You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/01/21 23:38:38 UTC

[01/50] [abbrv] calcite git commit: [CALCITE-1039] Assign a SqlKind value for each built-in aggregate function

Repository: calcite
Updated Branches:
  refs/heads/branch-release ba6e43c69 -> c4d346b0a


[CALCITE-1039] Assign a SqlKind value for each built-in aggregate function

Also, deprecate getParameterTypes and getReturnType methods in several aggregate functions.


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

Branch: refs/heads/branch-release
Commit: 8531ab11715109716215c0006d28014754e566f9
Parents: 808d540
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 4 02:47:17 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 5 15:16:35 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     | 22 +----
 .../org/apache/calcite/sql/SqlAggFunction.java  | 14 +++
 .../java/org/apache/calcite/sql/SqlKind.java    | 97 ++++++++++++++++++--
 .../org/apache/calcite/sql/SqlRankFunction.java | 17 ----
 .../calcite/sql/fun/SqlAvgAggFunction.java      | 48 ++++------
 .../calcite/sql/fun/SqlCountAggFunction.java    |  2 +-
 .../calcite/sql/fun/SqlCovarAggFunction.java    | 38 +++-----
 .../sql/fun/SqlFirstLastValueAggFunction.java   | 14 ++-
 .../sql/fun/SqlHistogramAggFunction.java        | 10 +-
 .../calcite/sql/fun/SqlLeadLagAggFunction.java  | 23 +++--
 .../calcite/sql/fun/SqlMinMaxAggFunction.java   | 49 ++++++----
 .../calcite/sql/fun/SqlNtileAggFunction.java    | 13 +--
 .../sql/fun/SqlSingleValueAggFunction.java      |  4 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    | 52 +++++------
 .../calcite/sql/fun/SqlSumAggFunction.java      |  4 +-
 .../sql/fun/SqlSumEmptyIsZeroAggFunction.java   |  2 +-
 16 files changed, 229 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/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 a7b3e08..e658749 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -64,11 +64,9 @@ import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.MultisetSqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -376,18 +374,9 @@ public abstract class RelOptUtil {
       // doing a min(TRUE)
 
       ret = createProject(ret, ImmutableList.of(extraExpr), null);
-      final List<RelDataType> argTypes =
-          ImmutableList.of(
-              typeFactory.createSqlType(SqlTypeName.BOOLEAN));
-
-      SqlAggFunction minFunction =
-          new SqlMinMaxAggFunction(
-              argTypes,
-              true,
-              SqlMinMaxAggFunction.MINMAX_COMPARABLE);
 
       final AggregateCall aggCall =
-          AggregateCall.create(minFunction,
+          AggregateCall.create(SqlStdOperatorTable.MIN,
               false,
               ImmutableList.of(0),
               -1,
@@ -457,15 +446,8 @@ public abstract class RelOptUtil {
 
       ret = createProject(ret, exprs, null);
 
-      final List<RelDataType> argTypes =
-          ImmutableList.of(typeFactory.createSqlType(SqlTypeName.BOOLEAN));
-
-      final SqlAggFunction minFunction =
-          new SqlMinMaxAggFunction(argTypes, true,
-              SqlMinMaxAggFunction.MINMAX_COMPARABLE);
-
       final AggregateCall aggCall =
-          AggregateCall.create(minFunction,
+          AggregateCall.create(SqlStdOperatorTable.MIN,
               false,
               ImmutableList.of(projectedKeyCount),
               -1,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/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 83fe248..d58987d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
@@ -17,12 +17,16 @@
 package org.apache.calcite.sql;
 
 import org.apache.calcite.plan.Context;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
+import java.util.List;
+
 /**
  * Abstract base class for the definition of an aggregate function: an operator
  * which aggregates sets of values into a result.
@@ -119,6 +123,16 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
   public final boolean requiresOver() {
     return requiresOver;
   }
+
+  @Deprecated // to be removed before 2.0
+  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before calcite-2.0");
+  }
+
+  @Deprecated // to be removed before 2.0
+  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before calcite-2.0");
+  }
 }
 
 // End SqlAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/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 5a8a2fa..78a9d7d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -625,6 +625,74 @@ public enum SqlKind {
   /** The internal {@code GROUP_ID()} function. */
   GROUP_ID,
 
+  // Aggregate functions
+
+  /** The {@code COUNT} aggregate function. */
+  COUNT,
+
+  /** The {@code SUM} aggregate function. */
+  SUM,
+
+  /** The {@code SUM0} aggregate function. */
+  SUM0,
+
+  /** The {@code MIN} aggregate function. */
+  MIN,
+
+  /** The {@code MAX} aggregate function. */
+  MAX,
+
+  /** The {@code LEAD} aggregate function. */
+  LEAD,
+
+  /** The {@code LAG} aggregate function. */
+  LAG,
+
+  /** The {@code FIRST_VALUE} aggregate function. */
+  FIRST_VALUE,
+
+  /** The {@code LAST_VALUE} aggregate function. */
+  LAST_VALUE,
+
+  /** The {@code COVAR_POP} aggregate function. */
+  COVAR_POP,
+
+  /** The {@code COVAR_SAMP} aggregate function. */
+  COVAR_SAMP,
+
+  /** The {@code REGR_SXX} aggregate function. */
+  REGR_SXX,
+
+  /** The {@code REGR_SYY} aggregate function. */
+  REGR_SYY,
+
+  /** The {@code AVG} aggregate function. */
+  AVG,
+
+  /** The {@code STDDEV_POP} aggregate function. */
+  STDDEV_POP,
+
+  /** The {@code STDDEV_SAMP} aggregate function. */
+  STDDEV_SAMP,
+
+  /** The {@code VAR_POP} aggregate function. */
+  VAR_POP,
+
+  /** The {@code VAR_SAMP} aggregate function. */
+  VAR_SAMP,
+
+  /** The {@code NTILE} aggregate function. */
+  NTILE,
+
+  /** The {@code COLLECT} aggregate function. */
+  COLLECT,
+
+  /** The {@code FUSION} aggregate function. */
+  FUSION,
+
+  /** The {@code SINGLE_VALUE} aggregate function. */
+  SINGLE_VALUE,
+
   // DDL and session control statements follow. The list is not exhaustive: feel
   // free to add more.
 
@@ -699,10 +767,21 @@ public enum SqlKind {
       EnumSet.of(UNION, INTERSECT, EXCEPT);
 
   /**
+   * Category consisting of all built-in aggregate functions.
+   */
+  public static final EnumSet<SqlKind> AGGREGATE =
+      EnumSet.of(COUNT, SUM, SUM0, MIN, MAX, LEAD, LAG, FIRST_VALUE,
+          LAST_VALUE, COVAR_POP, COVAR_SAMP, REGR_SXX, REGR_SYY,
+          AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP, NTILE, COLLECT,
+          FUSION, SINGLE_VALUE);
+
+  /**
    * Category consisting of all expression operators.
    *
    * <p>A node is an expression if it is NOT one of the following:
    * {@link #AS},
+   * {@link #ARGUMENT_ASSIGNMENT},
+   * {@link #DEFAULT},
    * {@link #DESCENDING},
    * {@link #SELECT},
    * {@link #JOIN},
@@ -715,17 +794,19 @@ public enum SqlKind {
    * {@link #FOLLOWING},
    * {@link #ORDER_BY},
    * {@link #COLLECTION_TABLE},
-   * {@link #TABLESAMPLE}.
+   * {@link #TABLESAMPLE},
+   * or an aggregate function.
    */
   public static final Set<SqlKind> EXPRESSION =
       EnumSet.complementOf(
-          EnumSet.of(
-              AS, ARGUMENT_ASSIGNMENT, DEFAULT,
-              DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND,
-              SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM, FLOOR, CEIL,
-              LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
-              NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
-              VALUES, WITH, WITH_ITEM));
+          concat(
+              EnumSet.of(AS, ARGUMENT_ASSIGNMENT, DEFAULT,
+                  DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND,
+                  SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM, FLOOR, CEIL,
+                  LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
+                  NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
+                  VALUES, WITH, WITH_ITEM),
+              AGGREGATE));
 
   /**
    * Category consisting of all DML operators.

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
index e2a2490..738add1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
@@ -16,23 +16,13 @@
  */
 package org.apache.calcite.sql;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
 /**
  * Operator which aggregates sets of values into a result.
  */
 public class SqlRankFunction extends SqlAggFunction {
-  //~ Instance fields --------------------------------------------------------
-
-  private final RelDataType type = null;
-
   //~ Constructors -----------------------------------------------------------
 
   public SqlRankFunction(String name, boolean requiresOrder) {
@@ -54,13 +44,6 @@ public class SqlRankFunction extends SqlAggFunction {
     return false;
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
-  }
 }
 
 // End SqlRankFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/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 b86d49c..95f8049 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
@@ -17,16 +17,13 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
+import com.google.common.base.Preconditions;
 
 /**
  * <code>Avg</code> is an aggregator which returns the average of the values
@@ -35,56 +32,49 @@ import java.util.List;
  * double</code>), and the result is the same type.
  */
 public class SqlAvgAggFunction extends SqlAggFunction {
-  //~ Instance fields --------------------------------------------------------
-
-  private final RelDataType type;
-  private final Subtype subtype;
-
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SqlAvgAggFunction
-   *
-   * @param type    Data type
-   * @param subtype Specific function, e.g. AVG or STDDEV_POP
+   * Creates a SqlAvgAggFunction.
    */
-  public SqlAvgAggFunction(
-      RelDataType type,
-      Subtype subtype) {
-    super(
-        subtype.name(),
+  public SqlAvgAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC,
         SqlFunctionCategory.NUMERIC,
         false,
         false);
-    this.type = type;
-    this.subtype = subtype;
+    Preconditions.checkArgument(kind == SqlKind.AVG
+        || kind == SqlKind.STDDEV_POP
+        || kind == SqlKind.STDDEV_SAMP
+        || kind == SqlKind.VAR_POP
+        || kind == SqlKind.VAR_SAMP);
   }
 
-  //~ Methods ----------------------------------------------------------------
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
+  @Deprecated // to be removed before 2.0
+  public SqlAvgAggFunction(
+      RelDataType type,
+      Subtype subtype) {
+    this(SqlKind.valueOf(subtype.name()));
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
+    //~ Methods ----------------------------------------------------------------
 
   /**
    * Returns the specific function, e.g. AVG or STDDEV_POP.
    *
    * @return Subtype
    */
+  @Deprecated // to be removed before 2.0
   public Subtype getSubtype() {
-    return subtype;
+    return Subtype.valueOf(kind.name());
   }
 
   /** Sub-type of aggregate function. */
+  @Deprecated // to be removed before 2.0
   public enum Subtype {
     AVG,
     STDDEV_POP,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
index e91f30b..dac4b21 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
@@ -48,7 +48,7 @@ public class SqlCountAggFunction extends SqlAggFunction {
     super(
         "COUNT",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.COUNT,
         ReturnTypes.BIGINT,
         null,
         SqlValidator.STRICT

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/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 df36095..ea23300 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
@@ -17,16 +17,13 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
+import com.google.common.base.Preconditions;
 
 /**
  * <code>Covar</code> is an aggregator which returns the Covariance of the
@@ -37,53 +34,48 @@ import java.util.List;
 public class SqlCovarAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
-  private final RelDataType type;
-  private final Subtype subtype;
-
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Creates a SqlCovarAggFunction.
-   *
-   * @param type    Data type
-   * @param subtype Specific function, e.g. COVAR_POP
    */
-  public SqlCovarAggFunction(RelDataType type, Subtype subtype) {
-    super(subtype.name(),
+  public SqlCovarAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC_NUMERIC,
         SqlFunctionCategory.NUMERIC,
         false,
         false);
-    this.type = type;
-    this.subtype = subtype;
+    Preconditions.checkArgument(kind == SqlKind.COVAR_POP
+        || kind == SqlKind.COVAR_SAMP
+        || kind == SqlKind.REGR_SXX
+        || kind == SqlKind.REGR_SYY);
   }
 
-  //~ Methods ----------------------------------------------------------------
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
+  @Deprecated // to be removed before 2.0
+  public SqlCovarAggFunction(RelDataType type, Subtype subtype) {
+    this(SqlKind.valueOf(subtype.name()));
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
+  //~ Methods ----------------------------------------------------------------
 
   /**
    * Returns the specific function, e.g. COVAR_POP or COVAR_SAMP.
    *
    * @return Subtype
    */
+  @Deprecated // to be removed before 2.0
   public Subtype getSubtype() {
-    return subtype;
+    return Subtype.valueOf(kind.name());
   }
 
   /**
    * Enum for defining specific types.
    */
+  @Deprecated // to be removed before 2.0
   public enum Subtype {
     COVAR_POP,
     COVAR_SAMP,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
index bc75424..edc6499 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -37,17 +38,24 @@ import java.util.List;
 public class SqlFirstLastValueAggFunction extends SqlAggFunction {
   //~ Constructors -----------------------------------------------------------
 
-  public SqlFirstLastValueAggFunction(boolean firstFlag) {
+  public SqlFirstLastValueAggFunction(SqlKind kind) {
     super(
-        firstFlag ? "FIRST_VALUE" : "LAST_VALUE",
+        kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.ANY,
         SqlFunctionCategory.NUMERIC,
         false,
         true);
+    Preconditions.checkArgument(kind == SqlKind.FIRST_VALUE
+        || kind == SqlKind.LAST_VALUE);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public SqlFirstLastValueAggFunction(boolean firstFlag) {
+    this(firstFlag ? SqlKind.FIRST_VALUE : SqlKind.LAST_VALUE);
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
index 8572338..1ba3821 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
@@ -29,15 +29,16 @@ import com.google.common.collect.ImmutableList;
 import java.util.List;
 
 /**
- * <code>HistogramAgg</code> is base operator which supports the Histogram
- * MIN/MAX aggregatoin functions. which returns the sum of the values which go
+ * <code>HISTOGRAM</code> is the base operator that supports the Histogram
+ * MIN/MAX aggregate functions. It returns the sum of the values which go
  * into it. It has precisely one argument of numeric type (<code>int</code>,
- * <code>long</code>, <code>float</code>, <code>double</code>) results are
- * retrived with (<code>HistogramMin</code>) and (<code>HistogramMax</code>)
+ * <code>long</code>, <code>float</code>, <code>double</code>); results are
+ * retrieved using (<code>HistogramMin</code>) and (<code>HistogramMax</code>).
  */
 public class SqlHistogramAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -62,6 +63,7 @@ public class SqlHistogramAggFunction extends SqlAggFunction {
     return ImmutableList.of(type);
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
index a2b0c45..5e5498b 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
@@ -31,6 +30,7 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeTransform;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -73,30 +73,29 @@ public class SqlLeadLagAggFunction extends SqlAggFunction {
         }
       });
 
-  public SqlLeadLagAggFunction(boolean isLead) {
-    super(
-        isLead ? "LEAD" : "LAG",
+  public SqlLeadLagAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         RETURN_TYPE,
         null,
         OPERAND_TYPES,
         SqlFunctionCategory.NUMERIC,
         false,
         true);
+    Preconditions.checkArgument(kind == SqlKind.LEAD
+        || kind == SqlKind.LAG);
   }
 
-  @Override public boolean allowsFraming() {
-    return false;
+  @Deprecated // to be removed before 2.0
+  public SqlLeadLagAggFunction(boolean isLead) {
+    this(isLead ? SqlKind.LEAD : SqlKind.LAG);
   }
 
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
+  @Override public boolean allowsFraming() {
+    return false;
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
 }
 
 // End SqlLeadLagAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
index 27a1a8b..391c09a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
@@ -26,6 +26,9 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 
 /**
@@ -57,62 +60,72 @@ public class SqlMinMaxAggFunction extends SqlAggFunction {
 
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public final List<RelDataType> argTypes;
-  private final boolean isMin;
-  private final int kind;
+  private final int minMaxKind;
 
   //~ Constructors -----------------------------------------------------------
 
-  public SqlMinMaxAggFunction(
-      List<RelDataType> argTypes,
-      boolean isMin,
-      int kind) {
-    super(
-        isMin ? "MIN" : "MAX",
+  /** Creates a SqlMinMaxAggFunction. */
+  public SqlMinMaxAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.COMPARABLE_ORDERED,
         SqlFunctionCategory.SYSTEM,
         false,
         false);
-    this.argTypes = argTypes;
-    this.isMin = isMin;
-    this.kind = kind;
+    this.argTypes = ImmutableList.of();
+    this.minMaxKind = MINMAX_COMPARABLE;
+    Preconditions.checkArgument(kind == SqlKind.MIN
+        || kind == SqlKind.MAX);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public SqlMinMaxAggFunction(
+      List<RelDataType> argTypes,
+      boolean isMin,
+      int minMaxKind) {
+    this(isMin ? SqlKind.MIN : SqlKind.MAX);
+    assert argTypes.isEmpty();
+    assert minMaxKind == MINMAX_COMPARABLE;
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public boolean isMin() {
-    return isMin;
+    return kind == SqlKind.MIN;
   }
 
+  @Deprecated // to be removed before 2.0
   public int getMinMaxKind() {
-    return kind;
+    return minMaxKind;
   }
 
   public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    switch (kind) {
+    switch (minMaxKind) {
     case MINMAX_PRIMITIVE:
     case MINMAX_COMPARABLE:
       return argTypes;
     case MINMAX_COMPARATOR:
       return argTypes.subList(1, 2);
     default:
-      throw Util.newInternal("bad kind: " + kind);
+      throw Util.newInternal("bad kind: " + minMaxKind);
     }
   }
 
   public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    switch (kind) {
+    switch (minMaxKind) {
     case MINMAX_PRIMITIVE:
     case MINMAX_COMPARABLE:
       return argTypes.get(0);
     case MINMAX_COMPARATOR:
       return argTypes.get(1);
     default:
-      throw Util.newInternal("bad kind: " + kind);
+      throw Util.newInternal("bad kind: " + minMaxKind);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
index 524e7d6..55eb8cb 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
@@ -16,16 +16,12 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import java.util.List;
-
 /**
  * <code>NTILE</code> aggregate function
  * return the value of given expression evaluated at given offset.
@@ -35,7 +31,7 @@ public class SqlNtileAggFunction extends SqlAggFunction {
     super(
         "NTILE",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.NTILE,
         ReturnTypes.INTEGER,
         null,
         OperandTypes.POSITIVE_INTEGER_LITERAL,
@@ -44,13 +40,6 @@ public class SqlNtileAggFunction extends SqlAggFunction {
         true);
   }
 
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
-
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
 }
 
 // End SqlNtileAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
index a27e180..c0c977c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
@@ -35,6 +35,7 @@ import java.util.List;
 public class SqlSingleValueAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -44,7 +45,7 @@ public class SqlSingleValueAggFunction extends SqlAggFunction {
     super(
         "SINGLE_VALUE",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SINGLE_VALUE,
         ReturnTypes.ARG0,
         null,
         OperandTypes.ANY,
@@ -64,6 +65,7 @@ public class SqlSingleValueAggFunction extends SqlAggFunction {
     return type;
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/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 fc524c6..3311f50 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
@@ -16,7 +16,6 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlAsOperator;
 import org.apache.calcite.sql.SqlBinaryOperator;
@@ -51,8 +50,6 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlModality;
 
-import com.google.common.collect.ImmutableList;
-
 /**
  * Implementation of {@link org.apache.calcite.sql.SqlOperatorTable} containing
  * the standard operators and functions.
@@ -688,43 +685,37 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <code>MIN</code> aggregate function.
    */
   public static final SqlAggFunction MIN =
-      new SqlMinMaxAggFunction(
-          ImmutableList.<RelDataType>of(),
-          true,
-          SqlMinMaxAggFunction.MINMAX_COMPARABLE);
+      new SqlMinMaxAggFunction(SqlKind.MIN);
 
   /**
    * <code>MAX</code> aggregate function.
    */
   public static final SqlAggFunction MAX =
-      new SqlMinMaxAggFunction(
-          ImmutableList.<RelDataType>of(),
-          false,
-          SqlMinMaxAggFunction.MINMAX_COMPARABLE);
+      new SqlMinMaxAggFunction(SqlKind.MAX);
 
   /**
    * <code>LAST_VALUE</code> aggregate function.
    */
   public static final SqlAggFunction LAST_VALUE =
-      new SqlFirstLastValueAggFunction(false);
+      new SqlFirstLastValueAggFunction(SqlKind.LAST_VALUE);
 
   /**
    * <code>FIRST_VALUE</code> aggregate function.
    */
   public static final SqlAggFunction FIRST_VALUE =
-      new SqlFirstLastValueAggFunction(true);
+      new SqlFirstLastValueAggFunction(SqlKind.FIRST_VALUE);
 
   /**
    * <code>LEAD</code> aggregate function.
    */
   public static final SqlAggFunction LEAD =
-      new SqlLeadLagAggFunction(true);
+      new SqlLeadLagAggFunction(SqlKind.LEAD);
 
   /**
    * <code>LAG</code> aggregate function.
    */
   public static final SqlAggFunction LAG =
-      new SqlLeadLagAggFunction(false);
+      new SqlLeadLagAggFunction(SqlKind.LAG);
 
   /**
    * <code>NTILE</code> aggregate function.
@@ -742,55 +733,55 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <code>AVG</code> aggregate function.
    */
   public static final SqlAggFunction AVG =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.AVG);
+      new SqlAvgAggFunction(SqlKind.AVG);
 
   /**
    * <code>STDDEV_POP</code> aggregate function.
    */
   public static final SqlAggFunction STDDEV_POP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.STDDEV_POP);
+      new SqlAvgAggFunction(SqlKind.STDDEV_POP);
 
   /**
    * <code>REGR_SXX</code> aggregate function.
    */
   public static final SqlAggFunction REGR_SXX =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.REGR_SXX);
+      new SqlCovarAggFunction(SqlKind.REGR_SXX);
 
   /**
    * <code>REGR_SYY</code> aggregate function.
    */
   public static final SqlAggFunction REGR_SYY =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.REGR_SYY);
+      new SqlCovarAggFunction(SqlKind.REGR_SYY);
 
   /**
    * <code>COVAR_POP</code> aggregate function.
    */
   public static final SqlAggFunction COVAR_POP =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.COVAR_POP);
+      new SqlCovarAggFunction(SqlKind.COVAR_POP);
 
   /**
    * <code>COVAR_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction COVAR_SAMP =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.COVAR_SAMP);
+      new SqlCovarAggFunction(SqlKind.COVAR_SAMP);
 
   /**
    * <code>STDDEV_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction STDDEV_SAMP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.STDDEV_SAMP);
+      new SqlAvgAggFunction(SqlKind.STDDEV_SAMP);
 
   /**
    * <code>VAR_POP</code> aggregate function.
    */
   public static final SqlAggFunction VAR_POP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.VAR_POP);
+      new SqlAvgAggFunction(SqlKind.VAR_POP);
 
   /**
    * <code>VAR_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction VAR_SAMP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.VAR_SAMP);
+      new SqlAvgAggFunction(SqlKind.VAR_SAMP);
 
   //-------------------------------------------------------------
   // WINDOW Aggregate Functions
@@ -1509,24 +1500,25 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    */
   public static final SqlAggFunction COLLECT =
       new SqlAggFunction("COLLECT",
-          SqlKind.OTHER_FUNCTION,
+          null,
+          SqlKind.COLLECT,
           ReturnTypes.TO_MULTISET,
           null,
           OperandTypes.ANY,
-          SqlFunctionCategory.SYSTEM) {
+          SqlFunctionCategory.SYSTEM, false, false) {
       };
 
   /**
    * The FUSION operator. Multiset aggregator function.
    */
   public static final SqlFunction FUSION =
-      new SqlFunction(
-          "FUSION",
-          SqlKind.OTHER_FUNCTION,
+      new SqlAggFunction("FUSION", null,
+          SqlKind.FUSION,
           ReturnTypes.ARG0,
           null,
           OperandTypes.MULTISET,
-          SqlFunctionCategory.SYSTEM);
+          SqlFunctionCategory.SYSTEM, false, false) {
+      };
 
   /**
    * The sequence next value function: <code>NEXT VALUE FOR sequence</code>

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
index 48e8ea4..5bf124d 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
@@ -39,6 +39,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
 
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -47,7 +48,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
     super(
         "SUM",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SUM,
         ReturnTypes.AGG_SUM,
         null,
         OperandTypes.NUMERIC,
@@ -63,6 +64,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
     return ImmutableList.of(type);
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
index 5b673e2..e8f7844 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
@@ -42,7 +42,7 @@ public class SqlSumEmptyIsZeroAggFunction extends SqlAggFunction {
   SqlSumEmptyIsZeroAggFunction() {
     super("$SUM0",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SUM0,
         ReturnTypes.AGG_SUM_EMPTY_IS_ZERO,
         null,
         OperandTypes.NUMERIC,


[29/50] [abbrv] calcite git commit: [CALCITE-794] Detect cycles when computing statistics

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
index f3cfae4..64ce699 100644
--- a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
+++ b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.runtime;
 
+import org.apache.calcite.util.ImmutableNullableList;
+
 import java.util.AbstractList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -53,7 +55,7 @@ public class FlatLists {
    * @param <T> Element type
    * @return List containing the given members
    */
-  public static <T> List<T> of(T... t) {
+  public static <T extends Comparable> List<T> of(T... t) {
     return flatList_(t, false);
   }
 
@@ -65,21 +67,54 @@ public class FlatLists {
    * @param <T> Element type
    * @return List containing the given members
    */
+  @Deprecated // to be removed before 2.0
   public static <T> List<T> copy(T... t) {
+    return flatListNotComparable(t);
+  }
+
+  /**
+   * Creates a memory-, CPU- and cache-efficient comparable immutable list,
+   * always copying the contents.
+   *
+   * <p>The elements are comparable, and so is the returned list.
+   * Elements may be null.
+   *
+   * @param t Array of members of list
+   * @param <T> Element type
+   * @return List containing the given members
+   */
+  public static <T extends Comparable> List<T> copyOf(T... t) {
     return flatList_(t, true);
   }
 
   /**
-   * Creates a memory-, CPU- and cache-efficient immutable list, optionally
-   * copying the list.
+   * Creates a memory-, CPU- and cache-efficient immutable list,
+   * always copying the contents.
+   *
+   * <p>The elements need not be comparable,
+   * and the returned list may not implement {@link Comparable}.
+   * Elements may be null.
+   *
+   * @param t Array of members of list
+   * @param <T> Element type
+   * @return List containing the given members
+   */
+  public static <T> List<T> copyOf(T... t) {
+    return flatListNotComparable(t);
+  }
+
+  /**
+   * Creates a memory-, CPU- and cache-efficient comparable immutable list,
+   * optionally copying the list.
    *
    * @param copy Whether to always copy the list
    * @param t Array of members of list
    * @return List containing the given members
    */
-  private static <T> List<T> flatList_(T[] t, boolean copy) {
+  private static <T extends Comparable> List<T> flatList_(T[] t, boolean copy) {
     switch (t.length) {
     case 0:
+      //noinspection unchecked
       return COMPARABLE_EMPTY_LIST;
     case 1:
       return Collections.singletonList(t[0]);
@@ -92,14 +127,37 @@ public class FlatLists {
       //   write our own implementation and reduce creation overhead a
       //   bit.
       if (copy) {
-        return new ComparableListImpl(Arrays.asList(t.clone()));
+        return new ComparableListImpl<>(Arrays.asList(t.clone()));
       } else {
-        return new ComparableListImpl(Arrays.asList(t));
+        return new ComparableListImpl<>(Arrays.asList(t));
       }
     }
   }
 
   /**
+   * Creates a memory-, CPU- and cache-efficient immutable list,
+   * always copying the list.
+   *
+   * @param t Array of members of list
+   * @return List containing the given members
+   */
+  private static <T> List<T> flatListNotComparable(T[] t) {
+    switch (t.length) {
+    case 0:
+      //noinspection unchecked
+      return COMPARABLE_EMPTY_LIST;
+    case 1:
+      return Collections.singletonList(t[0]);
+    case 2:
+      return new Flat2List<>(t[0], t[1]);
+    case 3:
+      return new Flat3List<>(t[0], t[1], t[2]);
+    default:
+      return ImmutableNullableList.copyOf(t);
+    }
+  }
+
+  /**
    * Creates a memory-, CPU- and cache-efficient immutable list from an
    * existing list. The list is always copied.
    *
@@ -110,13 +168,14 @@ public class FlatLists {
   public static <T> List<T> of(List<T> t) {
     switch (t.size()) {
     case 0:
+      //noinspection unchecked
       return COMPARABLE_EMPTY_LIST;
     case 1:
       return Collections.singletonList(t.get(0));
     case 2:
-      return new Flat2List<T>(t.get(0), t.get(1));
+      return new Flat2List<>(t.get(0), t.get(1));
     case 3:
-      return new Flat3List<T>(t.get(0), t.get(1), t.get(2));
+      return new Flat3List<>(t.get(0), t.get(1), t.get(2));
     default:
       // REVIEW: AbstractList contains a modCount field; we could
       //   write our own implementation and reduce creation overhead a

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java b/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java
index 25e7480..27f3cf8 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/StarTable.java
@@ -25,6 +25,7 @@ import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptUtil;
 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 org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.Schema;
@@ -129,7 +130,8 @@ public class StarTable extends AbstractTable implements TranslatableTable {
       super(cluster, cluster.traitSetOf(Convention.NONE), relOptTable);
     }
 
-    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeInfiniteCost();
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 e1d9f93..ba196db 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -48,6 +48,7 @@ import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rules.FilterCorrelateRule;
 import org.apache.calcite.rel.rules.FilterJoinRule;
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
@@ -1741,8 +1742,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
         // The join filters out the nulls.  So, it's ok if there are
         // nulls in the join keys.
-        if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
-            right,
+        final RelMetadataQuery mq = RelMetadataQuery.instance();
+        if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(mq, right,
             rightJoinKeys)) {
           SQL2REL_LOGGER.fine(rightJoinKeys.toString()
               + "are not unique keys for "
@@ -1956,8 +1957,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
         // The join filters out the nulls.  So, it's ok if there are
         // nulls in the join keys.
-        if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
-            left,
+        final RelMetadataQuery mq = RelMetadataQuery.instance();
+        if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(mq, left,
             correlatedInputRefJoinKeys)) {
           SQL2REL_LOGGER.fine(correlatedJoinKeys.toString()
               + "are not unique keys for "
@@ -2035,9 +2036,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
         // leftInputRel contains unique keys
         // i.e. each row is distinct and can group by on all the left
         // fields
-        if (!RelMdUtil.areColumnsDefinitelyUnique(
-            left,
-            allCols)) {
+        final RelMetadataQuery mq = RelMetadataQuery.instance();
+        if (!RelMdUtil.areColumnsDefinitelyUnique(mq, left, allCols)) {
           SQL2REL_LOGGER.fine("There are no unique keys for " + left);
           return;
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 8638df0..bd62c87 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -70,10 +70,8 @@ import com.google.common.collect.Iterables;
 import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.logging.Level;
 
@@ -107,7 +105,6 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   private final ReflectUtil.MethodDispatcher<TrimResult> trimFieldsDispatcher;
   private final RelBuilder relBuilder;
-  private Map<RelNode, Mapping> map = new HashMap<>();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -189,8 +186,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final ImmutableBitSet.Builder fieldsUsedBuilder = fieldsUsed.rebuild();
 
     // Fields that define the collation cannot be discarded.
-    final ImmutableList<RelCollation> collations =
-        RelMetadataQuery.collations(input);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final ImmutableList<RelCollation> collations = mq.collations(input);
     for (RelCollation collation : collations) {
       for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
         fieldsUsedBuilder.set(fieldCollation.getFieldIndex());
@@ -293,9 +290,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   }
 
   private TrimResult result(RelNode r, final Mapping mapping) {
-    map.put(r, mapping);
     final RexBuilder rexBuilder = relBuilder.getRexBuilder();
-    final RelNode r0 = r;
     for (final CorrelationId correlation : r.getVariablesSet()) {
       r = r.accept(
           new CorrelationReferenceFinder() {
@@ -362,7 +357,6 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final RelDataType rowType = project.getRowType();
     final int fieldCount = rowType.getFieldCount();
     final RelNode input = project.getInput();
-    final RelDataType inputRowType = input.getRowType();
 
     // Which fields are required from the input?
     final Set<RelDataTypeField> inputExtraFields =

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/util/Bug.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Bug.java b/core/src/main/java/org/apache/calcite/util/Bug.java
index 884b967..f60fbd1 100644
--- a/core/src/main/java/org/apache/calcite/util/Bug.java
+++ b/core/src/main/java/org/apache/calcite/util/Bug.java
@@ -171,9 +171,9 @@ public abstract class Bug {
   public static final boolean CALCITE_673_FIXED = false;
 
   /** Whether
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-794">[CALCITE-794]
-   * Detect cycles when computing statistics</a> is fixed. */
-  public static final boolean CALCITE_794_FIXED = false;
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1048">[CALCITE-1048]
+   * Make metadata more robust</a> is fixed. */
+  public static final boolean CALCITE_1048_FIXED = false;
 
   /** Whether
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1045">[CALCITE-1045]

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index c899cb8..9ea40b4 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -176,7 +176,7 @@ public enum BuiltInMethod {
   FUNCTION1_APPLY(Function1.class, "apply", Object.class),
   ARRAYS_AS_LIST(Arrays.class, "asList", Object[].class),
   ARRAY(SqlFunctions.class, "array", Object[].class),
-  LIST_N(FlatLists.class, "of", Object[].class),
+  LIST_N(FlatLists.class, "copyOf", Comparable[].class),
   LIST2(FlatLists.class, "of", Object.class, Object.class),
   LIST3(FlatLists.class, "of", Object.class, Object.class, Object.class),
   COMPARABLE_EMPTY_LIST(FlatLists.class, "COMPARABLE_EMPTY_LIST", true),
@@ -346,6 +346,7 @@ public enum BuiltInMethod {
   COLUMN_ORIGIN(ColumnOrigin.class, "getColumnOrigins", int.class),
   CUMULATIVE_COST(CumulativeCost.class, "getCumulativeCost"),
   NON_CUMULATIVE_COST(NonCumulativeCost.class, "getNonCumulativeCost"),
+  PREDICATES(Predicates.class, "getPredicates"),
   EXPLAIN_VISIBILITY(ExplainVisibility.class, "isVisibleInExplain",
       SqlExplainLevel.class),
   SCALAR_EXECUTE1(Scalar.class, "execute", Context.class),
@@ -353,7 +354,6 @@ public enum BuiltInMethod {
   CONTEXT_VALUES(Context.class, "values", true),
   CONTEXT_ROOT(Context.class, "root", true),
   DATA_CONTEXT_GET_QUERY_PROVIDER(DataContext.class, "getQueryProvider"),
-  PREDICATES(Predicates.class, "getPredicates"),
   METADATA_REL(Metadata.class, "rel");
 
   public final Method method;
@@ -373,7 +373,7 @@ public enum BuiltInMethod {
     MAP = builder.build();
   }
 
-  private BuiltInMethod(Method method, Constructor constructor, Field field) {
+  BuiltInMethod(Method method, Constructor constructor, Field field) {
     this.method = method;
     this.constructor = constructor;
     this.field = field;

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/util/NumberUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/NumberUtil.java b/core/src/main/java/org/apache/calcite/util/NumberUtil.java
index 4d5f71a..7e9d532 100644
--- a/core/src/main/java/org/apache/calcite/util/NumberUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/NumberUtil.java
@@ -155,6 +155,17 @@ public class NumberUtil {
 
     return a * b;
   }
+
+  /** Like {@link Math#min} but null safe. */
+  public static Double min(Double a, Double b) {
+    if (a == null) {
+      return b;
+    } else if (b == null) {
+      return a;
+    } else {
+      return Math.min(a, b);
+    }
+  }
 }
 
 // End NumberUtil.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java
index 8acd917..8d4dbd4 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java
@@ -110,8 +110,9 @@ public class TraitPropagationTest {
           RelOptUtil.dumpPlan("LOGICAL PLAN", planned, false,
               SqlExplainLevel.ALL_ATTRIBUTES));
     }
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     assertEquals("Sortedness was not propagated", 3,
-        RelMetadataQuery.getCumulativeCost(planned).getRows(), 0);
+        mq.getCumulativeCost(planned).getRows(), 0);
   }
 
   /**
@@ -306,7 +307,8 @@ public class TraitPropagationTest {
           groupSets, aggCalls);
     }
 
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeCost(1, 1, 1);
     }
   }
@@ -321,13 +323,14 @@ public class TraitPropagationTest {
     public static PhysProj create(final RelNode input,
         final List<RexNode> projects, RelDataType rowType) {
       final RelOptCluster cluster = input.getCluster();
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
       final RelTraitSet traitSet =
           cluster.traitSet().replace(PHYSICAL)
               .replaceIfs(
                   RelCollationTraitDef.INSTANCE,
                   new Supplier<List<RelCollation>>() {
                     public List<RelCollation> get() {
-                      return RelMdCollation.project(input, projects);
+                      return RelMdCollation.project(mq, input, projects);
                     }
                   });
       return new PhysProj(cluster, traitSet, input, projects, rowType);
@@ -338,7 +341,8 @@ public class TraitPropagationTest {
       return new PhysProj(getCluster(), traitSet, input, exps, rowType);
     }
 
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeCost(1, 1, 1);
     }
   }
@@ -359,7 +363,8 @@ public class TraitPropagationTest {
           offset, fetch);
     }
 
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeCost(1, 1, 1);
     }
   }
@@ -375,7 +380,8 @@ public class TraitPropagationTest {
           .add("i", integerType).build();
     }
 
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeCost(1, 1, 1);
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
index a44ab94..aea52f6 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
@@ -34,6 +34,7 @@ import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.convert.ConverterImpl;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -495,7 +496,8 @@ public class VolcanoPlannerTest {
     }
 
     // implement RelNode
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeInfiniteCost();
     }
 
@@ -523,7 +525,8 @@ public class VolcanoPlannerTest {
     }
 
     // implement RelNode
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeInfiniteCost();
     }
 
@@ -582,7 +585,8 @@ public class VolcanoPlannerTest {
     }
 
     // implement RelNode
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeTinyCost();
     }
 
@@ -605,7 +609,8 @@ public class VolcanoPlannerTest {
     }
 
     // implement RelNode
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeTinyCost();
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java
index e97a07c..c8530a2 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTraitTest.java
@@ -36,6 +36,7 @@ import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.convert.ConverterImpl;
 import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.util.Pair;
@@ -324,6 +325,7 @@ public class VolcanoPlannerTraitTest {
       RelTrait fromTrait = rel.getTraitSet().getTrait(this);
 
       if (conversionMap.containsKey(fromTrait)) {
+        final RelMetadataQuery mq = RelMetadataQuery.instance();
         for (Pair<RelTrait, ConverterRule> traitAndRule
             : conversionMap.get(fromTrait)) {
           RelTrait trait = traitAndRule.left;
@@ -332,7 +334,7 @@ public class VolcanoPlannerTraitTest {
           if (trait == toTrait) {
             RelNode converted = rule.convert(rel);
             if ((converted != null)
-                && (!planner.getCost(converted).isInfinite()
+                && (!planner.getCost(converted, mq).isInfinite()
                 || allowInfiniteCostConverters)) {
               return converted;
             }
@@ -369,8 +371,7 @@ public class VolcanoPlannerTraitTest {
       RelTrait fromTrait = converterRule.getInTrait();
       RelTrait toTrait = converterRule.getOutTrait();
 
-      conversionMap.put(fromTrait,
-          new Pair<RelTrait, ConverterRule>(toTrait, converterRule));
+      conversionMap.put(fromTrait, Pair.of(toTrait, converterRule));
     }
   }
 
@@ -391,7 +392,8 @@ public class VolcanoPlannerTraitTest {
     }
 
     // implement RelNode
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeInfiniteCost();
     }
 
@@ -437,7 +439,8 @@ public class VolcanoPlannerTraitTest {
     }
 
     // implement RelNode
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeTinyCost();
     }
 
@@ -454,7 +457,8 @@ public class VolcanoPlannerTraitTest {
     }
 
     // implement RelNode
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeInfiniteCost();
     }
 
@@ -508,7 +512,8 @@ public class VolcanoPlannerTraitTest {
     }
 
     // implement RelNode
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeTinyCost();
     }
 
@@ -731,7 +736,8 @@ public class VolcanoPlannerTraitTest {
           label);
     }
 
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       return planner.getCostFactory().makeZeroCost();
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index 51a4a70..8f4e5f3 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -214,26 +214,24 @@ public class JdbcAdapterTest {
             + "inner join scott.salgrade s \n"
             + "on e.sal > s.losal and e.sal < s.hisal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$2], ENAME=[$3], DNAME=[$1], GRADE=[$6])\n"
-            + "    JdbcJoin(condition=[=($5, $0)], joinType=[inner])\n"
-            + "      JdbcProject(DEPTNO=[$0], DNAME=[$1])\n"
-            + "        JdbcTableScan(table=[[SCOTT, DEPT]])\n"
-            + "      JdbcJoin(condition=[AND(>($2, $5), <($2, $6))], joinType=[inner])\n"
+            + "  JdbcProject(EMPNO=[$3], ENAME=[$4], DNAME=[$8], GRADE=[$0])\n"
+            + "    JdbcJoin(condition=[AND(>($5, $1), <($5, $2))], joinType=[inner])\n"
+            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])\n"
+            + "      JdbcJoin(condition=[=($3, $4)], joinType=[inner])\n"
             + "        JdbcProject(EMPNO=[$0], ENAME=[$1], SAL=[$5], DEPTNO=[$7])\n"
             + "          JdbcTableScan(table=[[SCOTT, EMP]])\n"
-            + "        JdbcTableScan(table=[[SCOTT, SALGRADE]])")
+            + "        JdbcProject(DEPTNO=[$0], DNAME=[$1])\n"
+            + "          JdbcTableScan(table=[[SCOTT, DEPT]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
-        .planHasSql("SELECT \"t0\".\"EMPNO\", \"t0\".\"ENAME\", "
-            + "\"t\".\"DNAME\", \"SALGRADE\".\"GRADE\"\n"
-            + "FROM (SELECT \"DEPTNO\", \"DNAME\"\n"
-            + "FROM \"SCOTT\".\"DEPT\") AS \"t\"\n"
+        .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
+            + "\"t0\".\"DNAME\", \"SALGRADE\".\"GRADE\"\n"
+            + "FROM \"SCOTT\".\"SALGRADE\"\n"
             + "INNER JOIN ((SELECT \"EMPNO\", \"ENAME\", \"SAL\", \"DEPTNO\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t0\"\n"
-            + "INNER JOIN \"SCOTT\".\"SALGRADE\" "
-            + "ON \"t0\".\"SAL\" > \"SALGRADE\".\"LOSAL\" "
-            + "AND \"t0\".\"SAL\" < \"SALGRADE\".\"HISAL\") "
-            + "ON \"t\".\"DEPTNO\" = \"t0\".\"DEPTNO\"");
+            + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
+            + "INNER JOIN (SELECT \"DEPTNO\", \"DNAME\"\n"
+            + "FROM \"SCOTT\".\"DEPT\") AS \"t0\" ON \"t\".\"DEPTNO\" = \"t0\".\"DEPTNO\")"
+            + " ON \"SALGRADE\".\"LOSAL\" < \"t\".\"SAL\" AND \"SALGRADE\".\"HISAL\" > \"t\".\"SAL\"");
   }
 
   @Test public void testCrossJoinWithJoinKeyPlan() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 41a462a..0e10877 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -3060,8 +3060,8 @@ public class JdbcTest {
         .query("select \"store_id\", \"grocery_sqft\" from \"store\"\n"
             + "where \"store_id\" < 10\n"
             + "order by 1 fetch first 5 rows only")
-        .explainContains(""
-            + "PLAN=EnumerableCalc(expr#0..23=[{inputs}], store_id=[$t0], grocery_sqft=[$t16])\n"
+        .explainContains("PLAN="
+            + "EnumerableCalc(expr#0..23=[{inputs}], store_id=[$t0], grocery_sqft=[$t16])\n"
             + "  EnumerableLimit(fetch=[5])\n"
             + "    EnumerableCalc(expr#0..23=[{inputs}], expr#24=[10], expr#25=[<($t0, $t24)], proj#0..23=[{exprs}], $condition=[$t25])\n"
             + "      EnumerableTableScan(table=[[foodmart2, store]])\n")
@@ -4778,10 +4778,10 @@ public class JdbcTest {
               return new Function<String, Object>() {
                 public Object apply(String v) {
                   switch (v) {
-                  case "calcite794":
-                    return Bug.CALCITE_794_FIXED;
                   case "calcite1045":
                     return Bug.CALCITE_1045_FIXED;
+                  case "calcite1048":
+                    return Bug.CALCITE_1048_FIXED;
                   }
                   return null;
                 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 f8c4b2f..96cd23b 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -148,7 +148,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
       double expected,
       double epsilon) {
     RelNode rel = convertSql(sql);
-    Double result = RelMetadataQuery.getPercentageOriginalRows(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Double result = mq.getPercentageOriginalRows(rel);
     assertTrue(result != null);
     assertEquals(expected, result, epsilon);
   }
@@ -233,7 +234,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   private Set<RelColumnOrigin> checkColumnOrigin(String sql) {
     RelNode rel = convertSql(sql);
-    return RelMetadataQuery.getColumnOrigins(rel, 0);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return mq.getColumnOrigins(rel, 0);
   }
 
   private void checkNoColumnOrigin(String sql) {
@@ -435,7 +437,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
       String sql,
       double expected) {
     RelNode rel = convertSql(sql);
-    Double result = RelMetadataQuery.getRowCount(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Double result = mq.getRowCount(rel);
     assertThat(result, notNullValue());
     assertEquals(expected, result, 0d);
   }
@@ -444,7 +447,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
       String sql,
       double expected) {
     RelNode rel = convertSql(sql);
-    Double result = RelMetadataQuery.getMaxRowCount(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Double result = mq.getMaxRowCount(rel);
     assertThat(result, notNullValue());
     assertEquals(expected, result, 0d);
   }
@@ -658,7 +662,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
       String sql,
       double expected) {
     RelNode rel = convertSql(sql);
-    Double result = RelMetadataQuery.getSelectivity(rel, null);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Double result = mq.getSelectivity(rel, null);
     assertTrue(result != null);
     assertEquals(expected, result, EPSILON);
   }
@@ -702,7 +707,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
   private void checkRelSelectivity(
       RelNode rel,
       double expected) {
-    Double result = RelMetadataQuery.getSelectivity(rel, null);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Double result = mq.getSelectivity(rel, null);
     assertTrue(result != null);
     assertEquals(expected, result, EPSILON);
   }
@@ -746,7 +752,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
         new CachingRelMetadataProvider(
             rel.getCluster().getMetadataProvider(),
             rel.getCluster().getPlanner()));
-    Double result = RelMetadataQuery.getSelectivity(rel, null);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Double result = mq.getSelectivity(rel, null);
     assertThat(result,
         nearTo(DEFAULT_COMP_SELECTIVITY * DEFAULT_EQUAL_SELECTIVITY, EPSILON));
   }
@@ -754,20 +761,18 @@ public class RelMetadataTest extends SqlToRelTestBase {
   @Test public void testDistinctRowCountTable() {
     // no unique key information is available so return null
     RelNode rel = convertSql("select * from emp where deptno = 10");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     ImmutableBitSet groupKey =
         ImmutableBitSet.of(rel.getRowType().getFieldNames().indexOf("DEPTNO"));
-    Double result =
-        RelMetadataQuery.getDistinctRowCount(
-            rel, groupKey, null);
+    Double result = mq.getDistinctRowCount(rel, groupKey, null);
     assertThat(result, nullValue());
   }
 
   @Test public void testDistinctRowCountTableEmptyKey() {
     RelNode rel = convertSql("select * from emp where deptno = 10");
     ImmutableBitSet groupKey = ImmutableBitSet.of(); // empty key
-    Double result =
-        RelMetadataQuery.getDistinctRowCount(
-            rel, groupKey, null);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Double result = mq.getDistinctRowCount(rel, groupKey, null);
     assertThat(result, is(1D));
   }
 
@@ -775,11 +780,12 @@ public class RelMetadataTest extends SqlToRelTestBase {
    * and {@link RelMetadataQuery#areColumnsUnique(RelNode, ImmutableBitSet)}
    * return consistent results. */
   private void assertUniqueConsistent(RelNode rel) {
-    Set<ImmutableBitSet> uniqueKeys = RelMetadataQuery.getUniqueKeys(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final Set<ImmutableBitSet> uniqueKeys = mq.getUniqueKeys(rel);
     final ImmutableBitSet allCols =
         ImmutableBitSet.range(0, rel.getRowType().getFieldCount());
     for (ImmutableBitSet key : allCols.powerSet()) {
-      Boolean result2 = RelMetadataQuery.areColumnsUnique(rel, key);
+      Boolean result2 = mq.areColumnsUnique(rel, key);
       assertTrue(result2 == null || result2 == isUnique(uniqueKeys, key));
     }
   }
@@ -801,14 +807,16 @@ public class RelMetadataTest extends SqlToRelTestBase {
    * NullPointerException"</a>. */
   @Test public void testJoinUniqueKeys() {
     RelNode rel = convertSql("select * from emp join dept using (deptno)");
-    Set<ImmutableBitSet> result = RelMetadataQuery.getUniqueKeys(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Set<ImmutableBitSet> result = mq.getUniqueKeys(rel);
     assertThat(result.isEmpty(), is(true));
     assertUniqueConsistent(rel);
   }
 
   @Test public void testGroupByEmptyUniqueKeys() {
     RelNode rel = convertSql("select count(*) from emp");
-    Set<ImmutableBitSet> result = RelMetadataQuery.getUniqueKeys(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    Set<ImmutableBitSet> result = mq.getUniqueKeys(rel);
     assertThat(result,
         CoreMatchers.<Set<ImmutableBitSet>>equalTo(
             ImmutableSet.of(ImmutableBitSet.of())));
@@ -817,7 +825,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   @Test public void testGroupByEmptyHavingUniqueKeys() {
     RelNode rel = convertSql("select count(*) from emp where 1 = 1");
-    Set<ImmutableBitSet> result = RelMetadataQuery.getUniqueKeys(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final Set<ImmutableBitSet> result = mq.getUniqueKeys(rel);
     assertThat(result,
         CoreMatchers.<Set<ImmutableBitSet>>equalTo(
             ImmutableSet.of(ImmutableBitSet.of())));
@@ -827,7 +836,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
   @Test public void testGroupBy() {
     RelNode rel = convertSql("select deptno, count(*), sum(sal) from emp\n"
             + "group by deptno");
-    Set<ImmutableBitSet> result = RelMetadataQuery.getUniqueKeys(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final Set<ImmutableBitSet> result = mq.getUniqueKeys(rel);
     assertThat(result,
         CoreMatchers.<Set<ImmutableBitSet>>equalTo(
             ImmutableSet.of(ImmutableBitSet.of(0))));
@@ -838,7 +848,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
     RelNode rel = convertSql("select deptno from emp\n"
             + "union\n"
             + "select deptno from dept");
-    Set<ImmutableBitSet> result = RelMetadataQuery.getUniqueKeys(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final Set<ImmutableBitSet> result = mq.getUniqueKeys(rel);
     assertThat(result,
         CoreMatchers.<Set<ImmutableBitSet>>equalTo(
             ImmutableSet.of(ImmutableBitSet.of(0))));
@@ -859,22 +870,23 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
     // Top node is a filter. Its metadata uses getColType(RelNode, int).
     assertThat(rel, instanceOf(LogicalFilter.class));
-    assertThat(rel.metadata(ColType.class).getColType(0),
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    assertThat(rel.metadata(ColType.class, mq).getColType(0),
         equalTo("DEPTNO-rel"));
-    assertThat(rel.metadata(ColType.class).getColType(1),
+    assertThat(rel.metadata(ColType.class, mq).getColType(1),
         equalTo("EXPR$1-rel"));
 
     // Next node is an aggregate. Its metadata uses
     // getColType(LogicalAggregate, int).
     final RelNode input = rel.getInput(0);
     assertThat(input, instanceOf(LogicalAggregate.class));
-    assertThat(input.metadata(ColType.class).getColType(0),
+    assertThat(input.metadata(ColType.class, mq).getColType(0),
         equalTo("DEPTNO-agg"));
 
     // There is no caching. Another request causes another call to the provider.
     assertThat(buf.toString(), equalTo("[DEPTNO-rel, EXPR$1-rel, DEPTNO-agg]"));
     assertThat(buf.size(), equalTo(3));
-    assertThat(input.metadata(ColType.class).getColType(0),
+    assertThat(input.metadata(ColType.class, mq).getColType(0),
         equalTo("DEPTNO-agg"));
     assertThat(buf.size(), equalTo(4));
 
@@ -884,19 +896,19 @@ public class RelMetadataTest extends SqlToRelTestBase {
     rel.getCluster().setMetadataProvider(
         new CachingRelMetadataProvider(
             rel.getCluster().getMetadataProvider(), planner));
-    assertThat(input.metadata(ColType.class).getColType(0),
+    assertThat(input.metadata(ColType.class, mq).getColType(0),
         equalTo("DEPTNO-agg"));
     assertThat(buf.size(), equalTo(5));
-    assertThat(input.metadata(ColType.class).getColType(0),
+    assertThat(input.metadata(ColType.class, mq).getColType(0),
         equalTo("DEPTNO-agg"));
     assertThat(buf.size(), equalTo(5));
-    assertThat(input.metadata(ColType.class).getColType(1),
+    assertThat(input.metadata(ColType.class, mq).getColType(1),
         equalTo("EXPR$1-agg"));
     assertThat(buf.size(), equalTo(6));
-    assertThat(input.metadata(ColType.class).getColType(1),
+    assertThat(input.metadata(ColType.class, mq).getColType(1),
         equalTo("EXPR$1-agg"));
     assertThat(buf.size(), equalTo(6));
-    assertThat(input.metadata(ColType.class).getColType(0),
+    assertThat(input.metadata(ColType.class, mq).getColType(0),
         equalTo("DEPTNO-agg"));
     assertThat(buf.size(), equalTo(6));
 
@@ -904,10 +916,10 @@ public class RelMetadataTest extends SqlToRelTestBase {
     long timestamp = planner.getRelMetadataTimestamp(rel);
     assertThat(timestamp, equalTo(0L));
     ((MockRelOptPlanner) planner).setRelMetadataTimestamp(timestamp + 1);
-    assertThat(input.metadata(ColType.class).getColType(0),
+    assertThat(input.metadata(ColType.class, mq).getColType(0),
         equalTo("DEPTNO-agg"));
     assertThat(buf.size(), equalTo(7));
-    assertThat(input.metadata(ColType.class).getColType(0),
+    assertThat(input.metadata(ColType.class, mq).getColType(0),
         equalTo("DEPTNO-agg"));
     assertThat(buf.size(), equalTo(7));
   }
@@ -957,7 +969,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
                 rexBuilder.makeInputRef(empSort, 0),
                 rexBuilder.makeInputRef(empSort, 3)));
 
-    collations = RelMdCollation.project(empSort, projects);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    collations = RelMdCollation.project(mq, empSort, projects);
     assertThat(collations.size(), equalTo(1));
     assertThat(collations.get(0).getFieldCollations().size(), equalTo(2));
     assertThat(collations.get(0).getFieldCollations().get(0).getFieldIndex(),
@@ -986,12 +999,13 @@ public class RelMetadataTest extends SqlToRelTestBase {
       throw Throwables.propagate(e);
     }
     collations =
-        RelMdCollation.mergeJoin(project, deptSort, leftKeys, rightKeys);
+        RelMdCollation.mergeJoin(mq, project, deptSort, leftKeys,
+            rightKeys);
     assertThat(collations,
         equalTo(join.getTraitSet().getTraits(RelCollationTraitDef.INSTANCE)));
 
     // Values (empty)
-    collations = RelMdCollation.values(empTable.getRowType(),
+    collations = RelMdCollation.values(mq, empTable.getRowType(),
         ImmutableList.<ImmutableList<RexLiteral>>of());
     assertThat(collations.toString(),
         equalTo("[[0, 1, 2, 3, 4, 5, 6, 7, 8], "
@@ -1006,7 +1020,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
     final LogicalValues emptyValues =
         LogicalValues.createEmpty(cluster, empTable.getRowType());
-    assertThat(RelMetadataQuery.collations(emptyValues), equalTo(collations));
+    assertThat(mq.collations(emptyValues), equalTo(collations));
 
     // Values (non-empty)
     final RelDataType rowType = cluster.getTypeFactory().builder()
@@ -1024,13 +1038,13 @@ public class RelMetadataTest extends SqlToRelTestBase {
     addRow(tuples, rexBuilder, 1, 2, 0, 3);
     addRow(tuples, rexBuilder, 2, 3, 2, 2);
     addRow(tuples, rexBuilder, 3, 3, 1, 4);
-    collations = RelMdCollation.values(rowType, tuples.build());
+    collations = RelMdCollation.values(mq, rowType, tuples.build());
     assertThat(collations.toString(),
         equalTo("[[0, 1, 2, 3], [1, 3]]"));
 
     final LogicalValues values =
         LogicalValues.create(cluster, rowType, tuples.build());
-    assertThat(RelMetadataQuery.collations(values), equalTo(collations));
+    assertThat(mq.collations(values), equalTo(collations));
   }
 
   private void addRow(ImmutableList.Builder<ImmutableList<RexLiteral>> builder,
@@ -1073,10 +1087,11 @@ public class RelMetadataTest extends SqlToRelTestBase {
   private void checkAverageRowSize(RelOptCluster cluster, RelOptTable empTable,
       RelOptTable deptTable) {
     final RexBuilder rexBuilder = cluster.getRexBuilder();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final LogicalTableScan empScan = LogicalTableScan.create(cluster, empTable);
 
-    Double rowSize = RelMetadataQuery.getAverageRowSize(empScan);
-    List<Double> columnSizes = RelMetadataQuery.getAverageColumnSizes(empScan);
+    Double rowSize = mq.getAverageRowSize(empScan);
+    List<Double> columnSizes = mq.getAverageColumnSizes(empScan);
 
     assertThat(columnSizes.size(),
         equalTo(empScan.getRowType().getFieldCount()));
@@ -1087,8 +1102,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
     // Empty values
     final LogicalValues emptyValues =
         LogicalValues.createEmpty(cluster, empTable.getRowType());
-    rowSize = RelMetadataQuery.getAverageRowSize(emptyValues);
-    columnSizes = RelMetadataQuery.getAverageColumnSizes(emptyValues);
+    rowSize = mq.getAverageRowSize(emptyValues);
+    columnSizes = mq.getAverageColumnSizes(emptyValues);
     assertThat(columnSizes.size(),
         equalTo(emptyValues.getRowType().getFieldCount()));
     assertThat(columnSizes,
@@ -1108,8 +1123,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
     addRow(tuples, rexBuilder, 3, "2",          null);
     final LogicalValues values =
         LogicalValues.create(cluster, rowType, tuples.build());
-    rowSize = RelMetadataQuery.getAverageRowSize(values);
-    columnSizes = RelMetadataQuery.getAverageColumnSizes(values);
+    rowSize = mq.getAverageRowSize(values);
+    columnSizes = mq.getAverageColumnSizes(values);
     assertThat(columnSizes.size(),
         equalTo(values.getRowType().getFieldCount()));
     assertThat(columnSizes, equalTo(Arrays.asList(4.0, 8.0, 3.0)));
@@ -1119,8 +1134,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final LogicalUnion union =
         LogicalUnion.create(ImmutableList.<RelNode>of(empScan, emptyValues),
             true);
-    rowSize = RelMetadataQuery.getAverageRowSize(union);
-    columnSizes = RelMetadataQuery.getAverageColumnSizes(union);
+    rowSize = mq.getAverageRowSize(union);
+    columnSizes = mq.getAverageColumnSizes(union);
     assertThat(columnSizes.size(), equalTo(9));
     assertThat(columnSizes,
         equalTo(Arrays.asList(4.0, 40.0, 20.0, 4.0, 8.0, 4.0, 4.0, 4.0, 1.0)));
@@ -1134,8 +1149,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
             rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN,
                 rexBuilder.makeInputRef(deptScan, 0),
                 rexBuilder.makeExactLiteral(BigDecimal.TEN)));
-    rowSize = RelMetadataQuery.getAverageRowSize(filter);
-    columnSizes = RelMetadataQuery.getAverageColumnSizes(filter);
+    rowSize = mq.getAverageRowSize(filter);
+    columnSizes = mq.getAverageColumnSizes(filter);
     assertThat(columnSizes.size(), equalTo(2));
     assertThat(columnSizes, equalTo(Arrays.asList(4.0, 20.0)));
     assertThat(rowSize, equalTo(24.0));
@@ -1152,8 +1167,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
                 rexBuilder.makeCall(SqlStdOperatorTable.CHAR_LENGTH,
                     rexBuilder.makeInputRef(filter, 1))),
             (List<String>) null);
-    rowSize = RelMetadataQuery.getAverageRowSize(deptProject);
-    columnSizes = RelMetadataQuery.getAverageColumnSizes(deptProject);
+    rowSize = mq.getAverageRowSize(deptProject);
+    columnSizes = mq.getAverageColumnSizes(deptProject);
     assertThat(columnSizes.size(), equalTo(4));
     assertThat(columnSizes, equalTo(Arrays.asList(4.0, 20.0, 4.0, 4.0)));
     assertThat(rowSize, equalTo(32.0));
@@ -1162,8 +1177,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final LogicalJoin join =
         LogicalJoin.create(empScan, deptProject, rexBuilder.makeLiteral(true),
             ImmutableSet.<CorrelationId>of(), JoinRelType.INNER);
-    rowSize = RelMetadataQuery.getAverageRowSize(join);
-    columnSizes = RelMetadataQuery.getAverageColumnSizes(join);
+    rowSize = mq.getAverageRowSize(join);
+    columnSizes = mq.getAverageColumnSizes(join);
     assertThat(columnSizes.size(), equalTo(13));
     assertThat(columnSizes,
         equalTo(
@@ -1177,26 +1192,26 @@ public class RelMetadataTest extends SqlToRelTestBase {
             ImmutableList.<ImmutableBitSet>of(),
             ImmutableList.of(
                 AggregateCall.create(
-                    SqlStdOperatorTable.COUNT, false, ImmutableIntList.of(), -1,
-                    2, join, null, null)));
-    rowSize = RelMetadataQuery.getAverageRowSize(aggregate);
-    columnSizes = RelMetadataQuery.getAverageColumnSizes(aggregate);
+                    SqlStdOperatorTable.COUNT, false, ImmutableIntList.of(),
+                    -1, 2, join, null, null)));
+    rowSize = mq.getAverageRowSize(aggregate);
+    columnSizes = mq.getAverageColumnSizes(aggregate);
     assertThat(columnSizes.size(), equalTo(3));
     assertThat(columnSizes, equalTo(Arrays.asList(4.0, 20.0, 8.0)));
     assertThat(rowSize, equalTo(32.0));
 
     // Smoke test Parallelism and Memory metadata providers
-    assertThat(RelMetadataQuery.memory(aggregate), nullValue());
-    assertThat(RelMetadataQuery.cumulativeMemoryWithinPhase(aggregate),
+    assertThat(mq.memory(aggregate), nullValue());
+    assertThat(mq.cumulativeMemoryWithinPhase(aggregate),
         nullValue());
-    assertThat(RelMetadataQuery.cumulativeMemoryWithinPhaseSplit(aggregate),
+    assertThat(mq.cumulativeMemoryWithinPhaseSplit(aggregate),
         nullValue());
-    assertThat(RelMetadataQuery.isPhaseTransition(aggregate), is(false));
-    assertThat(RelMetadataQuery.splitCount(aggregate), is(1));
+    assertThat(mq.isPhaseTransition(aggregate), is(false));
+    assertThat(mq.splitCount(aggregate), is(1));
   }
 
   /** Unit test for
-   * {@link org.apache.calcite.rel.metadata.RelMdPredicates#getPredicates(SemiJoin)}. */
+   * {@link org.apache.calcite.rel.metadata.RelMdPredicates#getPredicates(SemiJoin, RelMetadataQuery)}. */
   @Test public void testPredicates() {
     final Project rel = (Project) convertSql("select * from emp, dept");
     final Join join = (Join) rel.getInput();
@@ -1216,10 +1231,11 @@ public class RelMetadataTest extends SqlToRelTestBase {
   private void checkPredicates(RelOptCluster cluster, RelOptTable empTable,
       RelOptTable deptTable) {
     final RexBuilder rexBuilder = cluster.getRexBuilder();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final LogicalTableScan empScan = LogicalTableScan.create(cluster, empTable);
 
     RelOptPredicateList predicates =
-        RelMetadataQuery.getPulledUpPredicates(empScan);
+        mq.getPulledUpPredicates(empScan);
     assertThat(predicates.pulledUpPredicates.isEmpty(), is(true));
 
     final LogicalFilter filter =
@@ -1229,7 +1245,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
                     empScan.getRowType().getFieldNames().indexOf("EMPNO")),
                 rexBuilder.makeExactLiteral(BigDecimal.ONE)));
 
-    predicates = RelMetadataQuery.getPulledUpPredicates(filter);
+    predicates = mq.getPulledUpPredicates(filter);
     assertThat(predicates.pulledUpPredicates.toString(), is("[=($0, 1)]"));
 
     final LogicalTableScan deptScan =
@@ -1253,7 +1269,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
             ImmutableIntList.of(rightDeptnoField.getIndex()
                     + empScan.getRowType().getFieldCount()));
 
-    predicates = RelMetadataQuery.getPulledUpPredicates(semiJoin);
+    predicates = mq.getPulledUpPredicates(semiJoin);
     assertThat(predicates.pulledUpPredicates, sortsAs("[=($0, 1)]"));
     assertThat(predicates.leftInferredPredicates, sortsAs("[]"));
     assertThat(predicates.rightInferredPredicates.isEmpty(), is(true));
@@ -1261,14 +1277,15 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   /**
    * Unit test for
-   * {@link org.apache.calcite.rel.metadata.RelMdPredicates#getPredicates(Aggregate)}.
+   * {@link org.apache.calcite.rel.metadata.RelMdPredicates#getPredicates(Aggregate, RelMetadataQuery)}.
    */
   @Test public void testPullUpPredicatesFromAggregation() {
     final String sql = "select a, max(b) from (\n"
         + "  select 1 as a, 2 as b from emp)subq\n"
         + "group by a";
     final Aggregate rel = (Aggregate) convertSql(sql);
-    RelOptPredicateList inputSet = RelMetadataQuery.getPulledUpPredicates(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    RelOptPredicateList inputSet = mq.getPulledUpPredicates(rel);
     ImmutableList<RexNode> pulledUpPredicates = inputSet.pulledUpPredicates;
     assertThat(pulledUpPredicates, sortsAs("[=($0, 1)]"));
   }
@@ -1279,7 +1296,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
         + "  from emp\n"
         + "  where mgr is null and deptno < 10)";
     final RelNode rel = convertSql(sql);
-    RelOptPredicateList list = RelMetadataQuery.getPulledUpPredicates(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    RelOptPredicateList list = mq.getPulledUpPredicates(rel);
     assertThat(list.pulledUpPredicates,
         sortsAs("[<($0, 10), =($3, 'y'), =($4, CAST('1'):INTEGER NOT NULL), "
             + "IS NULL($1), IS NULL($2)]"));
@@ -1290,7 +1308,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
         + "  from emp\n"
         + "  where mgr is null and deptno < 10";
     final RelNode rel = convertSql(sql);
-    RelOptPredicateList list = RelMetadataQuery.getPulledUpPredicates(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    RelOptPredicateList list = mq.getPulledUpPredicates(rel);
     // Uses "IS NOT DISTINCT FROM" rather than "=" because cannot guarantee not null.
     assertThat(list.pulledUpPredicates,
         sortsAs("[IS NOT DISTINCT FROM($0, CASE(=(1, 1), null, 1))]"));
@@ -1347,7 +1366,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
      * {@link org.apache.calcite.rel.logical.LogicalAggregate}, called via
      * reflection. */
     @SuppressWarnings("UnusedDeclaration")
-    public String getColType(Aggregate rel, int column) {
+    public String getColType(Aggregate rel, RelMetadataQuery mq, int column) {
       final String name =
           rel.getRowType().getFieldList().get(column).getName() + "-agg";
       THREAD_LIST.get().add(name);
@@ -1357,7 +1376,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     /** Implementation of {@link ColType#getColType(int)} for
      * {@link RelNode}, called via reflection. */
     @SuppressWarnings("UnusedDeclaration")
-    public String getColType(RelNode rel, int column) {
+    public String getColType(RelNode rel, RelMetadataQuery mq, int column) {
       final String name =
           rel.getRowType().getFieldList().get(column).getName() + "-rel";
       THREAD_LIST.get().add(name);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index fae8cb9..d604938 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -275,8 +275,8 @@ public class PlannerTest {
     SqlNode parse = planner.parse(sql);
     SqlNode validate = planner.validate(parse);
     RelNode rel = planner.rel(validate).project();
-    final RelOptPredicateList predicates =
-        RelMetadataQuery.getPulledUpPredicates(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final RelOptPredicateList predicates = mq.getPulledUpPredicates(rel);
     final String buf = predicates.pulledUpPredicates.toString();
     assertThat(buf, equalTo(expectedPredicates));
   }
@@ -562,7 +562,7 @@ public class PlannerTest {
         Programs.of(ruleSet2));
     SqlNode parse = planner.parse("select * from \"emps\"");
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).rel;
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -1071,7 +1071,8 @@ public class PlannerTest {
   @Test public void testMergeProjectForceMode() throws Exception {
     RuleSet ruleSet =
         RuleSets.ofList(
-            new ProjectMergeRule(true, RelFactories.DEFAULT_PROJECT_FACTORY));
+            new ProjectMergeRule(true,
+                RelBuilder.proto(RelFactories.DEFAULT_PROJECT_FACTORY)));
     Planner planner = getPlanner(null, Programs.of(ruleSet));
     planner.close();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 60122fb..8bedcbc 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -950,6 +950,16 @@ public class UtilTest {
     assertEquals(ab, ab0);
     assertEquals(ab.hashCode(), ab0.hashCode());
 
+    final List<String> abc = FlatLists.of("A", "B", "C");
+    final List<String> abc0 = Arrays.asList("A", "B", "C");
+    assertEquals(abc, abc0);
+    assertEquals(abc.hashCode(), abc0.hashCode());
+
+    final List<Object> abc1 = FlatLists.of((Object) "A", "B", "C");
+    assertEquals(abc1, abc0);
+    assertEquals(abc, abc0);
+    assertEquals(abc1.hashCode(), abc0.hashCode());
+
     final List<String> an = FlatLists.of("A", null);
     final List<String> an0 = Arrays.asList("A", null);
     assertEquals(an, an0);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 b76cc90..52a15cc 100644
--- a/core/src/test/resources/sql/agg.iq
+++ b/core/src/test/resources/sql/agg.iq
@@ -1393,7 +1393,7 @@ group by deptno, job;
 (3 rows)
 
 !ok
-!if (fixed.calcite794) {
+!if (fixed.calcite1048) {
 select job, sum(sal) as sum_sal, deptno
 from "scott".emp
 where deptno = 10

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java
index 45b4baf..8d2ae04 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoFilter.java
@@ -23,6 +23,7 @@ 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;
@@ -54,8 +55,9 @@ public class MongoFilter extends Filter implements MongoRel {
     assert getConvention() == child.getConvention();
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.1);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(0.1);
   }
 
   public MongoFilter copy(RelTraitSet traitSet, RelNode input,

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java
index 4e40788..d8a17ec 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoProject.java
@@ -23,6 +23,7 @@ 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;
@@ -56,8 +57,9 @@ public class MongoProject extends Project implements MongoRel {
         rowType);
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.1);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(0.1);
   }
 
   public void implement(Implementor implementor) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java
index c1bb26e..193e225 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoSort.java
@@ -24,6 +24,7 @@ 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;
@@ -44,8 +45,9 @@ public class MongoSort extends Sort implements MongoRel {
     assert getConvention() == child.getConvention();
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.05);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(0.05);
   }
 
   @Override public Sort copy(RelTraitSet traitSet, RelNode input,

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTableScan.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTableScan.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTableScan.java
index 36e220b..a47adee 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTableScan.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTableScan.java
@@ -24,6 +24,7 @@ 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;
@@ -66,11 +67,12 @@ public class MongoTableScan extends TableScan implements MongoRel {
     return projectRowType != null ? projectRowType : super.deriveRowType();
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // scans with a small project list are cheaper
     final float f = projectRowType == null ? 1f
         : (float) projectRowType.getFieldCount() / 100f;
-    return super.computeSelfCost(planner).multiplyBy(.1 * f);
+    return super.computeSelfCost(planner, mq).multiplyBy(.1 * f);
   }
 
   @Override public void register(RelOptPlanner planner) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java
index d6a4475..cbbeb86 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java
@@ -33,6 +33,7 @@ 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;
@@ -62,8 +63,9 @@ public class MongoToEnumerableConverter
         getCluster(), traitSet, sole(inputs));
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(.1);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(.1);
   }
 
   public Result implement(EnumerableRelImplementor implementor, Prefer pref) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/spark/src/main/java/org/apache/calcite/adapter/spark/EnumerableToSparkConverter.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/calcite/adapter/spark/EnumerableToSparkConverter.java b/spark/src/main/java/org/apache/calcite/adapter/spark/EnumerableToSparkConverter.java
index 11f7820..ae7aae0 100644
--- a/spark/src/main/java/org/apache/calcite/adapter/spark/EnumerableToSparkConverter.java
+++ b/spark/src/main/java/org/apache/calcite/adapter/spark/EnumerableToSparkConverter.java
@@ -31,6 +31,7 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 import java.util.List;
 
@@ -55,8 +56,9 @@ public class EnumerableToSparkConverter
         getCluster(), traitSet, sole(inputs));
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(.01);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(.01);
   }
 
   public Result implementSpark(Implementor implementor) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/spark/src/main/java/org/apache/calcite/adapter/spark/JdbcToSparkConverter.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/calcite/adapter/spark/JdbcToSparkConverter.java b/spark/src/main/java/org/apache/calcite/adapter/spark/JdbcToSparkConverter.java
index b919931..bca76c2 100644
--- a/spark/src/main/java/org/apache/calcite/adapter/spark/JdbcToSparkConverter.java
+++ b/spark/src/main/java/org/apache/calcite/adapter/spark/JdbcToSparkConverter.java
@@ -36,6 +36,7 @@ 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.sql.SqlDialect;
 import org.apache.calcite.util.BuiltInMethod;
 
@@ -59,8 +60,9 @@ public class JdbcToSparkConverter
         getCluster(), traitSet, sole(inputs));
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(.1);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(.1);
   }
 
   public SparkRel.Result implementSpark(SparkRel.Implementor implementor) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java b/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java
index 24199cb..2bea47b 100644
--- a/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java
+++ b/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java
@@ -43,8 +43,8 @@ import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rules.FilterToCalcRule;
 import org.apache.calcite.rel.rules.ProjectToCalcRule;
@@ -271,15 +271,15 @@ public abstract class SparkRules {
       return program.explainCalc(super.explainTerms(pw));
     }
 
-    public double getRows() {
-      return LogicalFilter.estimateFilteredRows(getInput(), program);
+    @Override public double estimateRowCount(RelMetadataQuery mq) {
+      return RelMdUtil.estimateFilteredRows(getInput(), program, mq);
     }
 
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      double dRows = RelMetadataQuery.getRowCount(this);
-      double dCpu =
-          RelMetadataQuery.getRowCount(getInput())
-              * program.getExprCount();
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      double dRows = mq.getRowCount(this);
+      double dCpu = mq.getRowCount(getInput())
+          * program.getExprCount();
       double dIo = 0;
       return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/spark/src/main/java/org/apache/calcite/adapter/spark/SparkToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/calcite/adapter/spark/SparkToEnumerableConverter.java b/spark/src/main/java/org/apache/calcite/adapter/spark/SparkToEnumerableConverter.java
index e270d44..b96dd5f 100644
--- a/spark/src/main/java/org/apache/calcite/adapter/spark/SparkToEnumerableConverter.java
+++ b/spark/src/main/java/org/apache/calcite/adapter/spark/SparkToEnumerableConverter.java
@@ -33,6 +33,7 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 import java.util.List;
 
@@ -59,8 +60,9 @@ public class SparkToEnumerableConverter
         getCluster(), traitSet, sole(inputs));
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(.01);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(.01);
   }
 
   public Result implement(EnumerableRelImplementor implementor, Prefer pref) {


[42/50] [abbrv] calcite git commit: [CALCITE-1053] CPU spin in ReflectiveRelMetadataProvider.apply -> HashMap.get

Posted by jh...@apache.org.
[CALCITE-1053] CPU spin in ReflectiveRelMetadataProvider.apply -> HashMap.get

The map in question is shared between multiple threads and updated at the same time.
Thus the map should be a ConcurrentMap.

Problem was reported here: https://mail-archives.apache.org/mod_mbox/calcite-dev/201601.mbox/%3C56952E8F.7090705%40gmail.com%3E


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

Branch: refs/heads/branch-release
Commit: 2712d7dafc1b724c33b0c29846666c1903d5afb2
Parents: 5323d8d
Author: Vladimir Sitnikov <si...@gmail.com>
Authored: Wed Jan 13 11:36:19 2016 +0300
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 13 08:42:52 2016 -0800

----------------------------------------------------------------------
 .../rel/metadata/ReflectiveRelMetadataProvider.java    | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/2712d7da/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
index beaedfe..8cdb55a 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
@@ -39,10 +39,11 @@ import java.lang.reflect.Proxy;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 /**
  * Implementation of the {@link RelMetadataProvider} interface that dispatches
@@ -60,7 +61,7 @@ public class ReflectiveRelMetadataProvider
     implements RelMetadataProvider, ReflectiveVisitor {
 
   //~ Instance fields --------------------------------------------------------
-  private final Map<Class<RelNode>, UnboundMetadata> map;
+  private final ConcurrentMap<Class<RelNode>, UnboundMetadata> map;
   private final Class<? extends Metadata> metadataClass0;
 
   //~ Constructors -----------------------------------------------------------
@@ -72,7 +73,7 @@ public class ReflectiveRelMetadataProvider
    * @param metadataClass0 Metadata class
    */
   protected ReflectiveRelMetadataProvider(
-      Map<Class<RelNode>, UnboundMetadata> map,
+      ConcurrentMap<Class<RelNode>, UnboundMetadata> map,
       Class<? extends Metadata> metadataClass0) {
     assert !map.isEmpty() : "are your methods named wrong?";
     this.map = map;
@@ -135,7 +136,11 @@ public class ReflectiveRelMetadataProvider
       }
     }
 
-    final Map<Class<RelNode>, UnboundMetadata> methodsMap = new HashMap<>();
+    // This needs to be a councurrent map since RelMetadataProvider are cached in static
+    // fields, thus the map is subject to concurrent modifications later.
+    // See map.put in org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider.apply(
+    // java.lang.Class<? extends org.apache.calcite.rel.RelNode>)
+    final ConcurrentMap<Class<RelNode>, UnboundMetadata> methodsMap = new ConcurrentHashMap<>();
     for (Class<RelNode> key : classes) {
       ImmutableNullableList.Builder<Method> builder =
           ImmutableNullableList.builder();


[14/50] [abbrv] calcite git commit: [CALCITE-1036] DiffRepository should not insert new resources at the end of the repository

Posted by jh...@apache.org.
[CALCITE-1036] DiffRepository should not insert new resources at the end of the repository


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

Branch: refs/heads/branch-release
Commit: 8bc5f8538f5c284e93ddc4ecfb59dab9458117f7
Parents: 9a52b5e
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Dec 23 16:18:22 2015 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/test/DiffRepository.java | 54 +++++++++++++++++---
 1 file changed, 47 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/8bc5f853/core/src/test/java/org/apache/calcite/test/DiffRepository.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/DiffRepository.java b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
index 78dcc10..90ecc83 100644
--- a/core/src/test/java/org/apache/calcite/test/DiffRepository.java
+++ b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.test;
 
 import org.apache.calcite.avatica.util.Spaces;
+import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.XmlOutput;
 
@@ -302,7 +303,7 @@ public class DiffRepository {
   private synchronized String get(
       final String testCaseName,
       String resourceName) {
-    Element testCaseElement = getTestCaseElement(testCaseName, true);
+    Element testCaseElement = getTestCaseElement(testCaseName, true, null);
     if (testCaseElement == null) {
       if (baseRepository != null) {
         return baseRepository.get(testCaseName, resourceName);
@@ -355,17 +356,18 @@ public class DiffRepository {
    */
   private synchronized Element getTestCaseElement(
       final String testCaseName,
-      boolean checkOverride) {
+      boolean checkOverride,
+      List<Pair<String, Element>> elements) {
     final NodeList childNodes = root.getChildNodes();
     for (int i = 0; i < childNodes.getLength(); i++) {
       Node child = childNodes.item(i);
       if (child.getNodeName().equals(TEST_CASE_TAG)) {
         Element testCase = (Element) child;
-        if (testCaseName.equals(
-            testCase.getAttribute(TEST_CASE_NAME_ATTR))) {
+        final String name = testCase.getAttribute(TEST_CASE_NAME_ATTR);
+        if (testCaseName.equals(name)) {
           if (checkOverride
               && (baseRepository != null)
-              && (baseRepository.getTestCaseElement(testCaseName, false) != null)
+              && (baseRepository.getTestCaseElement(testCaseName, false, null) != null)
               && !"true".equals(
                   testCase.getAttribute(TEST_CASE_OVERRIDES_ATTR))) {
             throw new RuntimeException(
@@ -375,6 +377,9 @@ public class DiffRepository {
           }
           return testCase;
         }
+        if (elements != null) {
+          elements.add(Pair.of(name, testCase));
+        }
       }
     }
     return null;
@@ -456,11 +461,13 @@ public class DiffRepository {
       String testCaseName,
       String resourceName,
       String value) {
-    Element testCaseElement = getTestCaseElement(testCaseName, true);
+    final List<Pair<String, Element>> map = new ArrayList<>();
+    Element testCaseElement = getTestCaseElement(testCaseName, true, map);
     if (testCaseElement == null) {
       testCaseElement = doc.createElement(TEST_CASE_TAG);
       testCaseElement.setAttribute(TEST_CASE_NAME_ATTR, testCaseName);
-      root.appendChild(testCaseElement);
+      Node refElement = ref(testCaseName, map);
+      root.insertBefore(testCaseElement, refElement);
     }
     Element resourceElement =
         getResourceElement(testCaseElement, resourceName, true);
@@ -479,6 +486,39 @@ public class DiffRepository {
     flushDoc();
   }
 
+  private Node ref(String testCaseName, List<Pair<String, Element>> map) {
+    if (map.isEmpty()) {
+      return null;
+    }
+    // Compute the position that the new element should be if the map were
+    // sorted.
+    int i = 0;
+    final List<String> names = Pair.left(map);
+    for (String s : names) {
+      if (s.compareToIgnoreCase(testCaseName) <= 0) {
+        ++i;
+      }
+    }
+    // Starting at a proportional position in the list,
+    // move forwards through lesser names, then
+    // move backwards through greater names.
+    //
+    // The intended effect is that if the list is already sorted, the new item
+    // will end up in exactly the right position, and if the list is not sorted,
+    // the new item will end up in approximately the right position.
+    while (i < map.size()
+        && names.get(i).compareToIgnoreCase(testCaseName) < 0) {
+      ++i;
+    }
+    if (i >= map.size() - 1) {
+      return null;
+    }
+    while (i >= 0 && names.get(i).compareToIgnoreCase(testCaseName) > 0) {
+      --i;
+    }
+    return map.get(i + 1).right;
+  }
+
   /**
    * Flushes the reference document to the file system.
    */


[48/50] [abbrv] calcite git commit: Add release notes for version 1.6.0

Posted by jh...@apache.org.
Add release notes for version 1.6.0


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

Branch: refs/heads/branch-release
Commit: df5ec6dd286ed436e52b89be36d56ba6ae66051e
Parents: 6257655
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Jan 14 14:17:56 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Jan 18 11:02:02 2016 -0800

----------------------------------------------------------------------
 README                |   2 +-
 pom.xml               |   2 +-
 site/_docs/history.md | 234 +++++++++++++++++++++++++++++++++++++++++++++
 site/_docs/howto.md   |   6 +-
 4 files changed, 239 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/df5ec6dd/README
----------------------------------------------------------------------
diff --git a/README b/README
index a68be99..b492f4d 100644
--- a/README
+++ b/README
@@ -1,4 +1,4 @@
-Apache Calcite release 1.5.0
+Apache Calcite release 1.6.0
 
 This is a source or binary distribution of Apache Calcite.
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/df5ec6dd/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2d83845..5c23392 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>5</version.minor>
+    <version.minor>6</version.minor>
   </properties>
 
   <issueManagement>

http://git-wip-us.apache.org/repos/asf/calcite/blob/df5ec6dd/site/_docs/history.md
----------------------------------------------------------------------
diff --git a/site/_docs/history.md b/site/_docs/history.md
index 6769758..9fc0042 100644
--- a/site/_docs/history.md
+++ b/site/_docs/history.md
@@ -28,6 +28,238 @@ For a full list of releases, see
 Downloads are available on the
 [downloads page]({{ site.baseurl }}/downloads/).
 
+## <a href="https://github.com/apache/calcite/releases/tag/calcite-1.6.0">1.6.0</a> / 2016-01-14
+{: #v1-6-0}
+
+As usual in this release, there are new SQL features, improvements to
+planning rules and Avatica, and lots of bug fixes. We'll spotlight a
+couple of features make it easier to handle complex queries.
+
+[<a href="https://issues.apache.org/jira/browse/CALCITE-816">CALCITE-816</a>]
+allows you to represent sub-queries (`EXISTS`, `IN` and scalar) as
+`RexSubQuery`, a kind of expression in the relational algebra. Until
+now, the sql-to-rel converter was burdened with expanding sub-queries,
+and people creating relational algebra directly (or via RelBuilder)
+could only create 'flat' relational expressions. Now we have planner
+rules to expand and de-correlate sub-queries.
+
+Metadata is the fuel that powers query planning. It includes
+traditional query-planning statistics such as cost and row-count
+estimates, but also information such as which columns form unique
+keys, unique and what predicates are known to apply to a relational
+expression's output rows. From the predicates we can deduce which
+columns are constant, and following
+[<a href="https://issues.apache.org/jira/browse/CALCITE-1023">CALCITE-1023</a>]
+we can now remove constant columns from `GROUP BY` keys.
+
+Metadata is often computed recursively, and it is hard to safely and
+efficiently calculate metadata on a graph of `RelNode`s that is large,
+frequently cyclic, and constantly changing.
+[<a href="https://issues.apache.org/jira/browse/CALCITE-794">CALCITE-794</a>]
+introduces a context to each metadata call. That context can detect
+cyclic metadata calls and produce a safe answer to the metadata
+request. It will also allow us to add finer-grained caching and
+further tune the metadata layer.
+
+New features
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-816">CALCITE-816</a>]
+  Represent sub-query as a `RexNode`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-854">CALCITE-854</a>]
+  Implement `UNNEST ... WITH ORDINALITY`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1003">CALCITE-1003</a>]
+  Utility to convert `RelNode` to SQL (Amogh Margoor)
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-1010">CALCITE-1010</a>]
+    `FETCH/LIMIT` and `OFFSET` in RelToSqlConverter (Amogh Margoor)
+  * Move code from `JdbcImplementor` and `JdbcRules` to new class
+    `SqlImplementor`
+  * Deduce dialect's null collation from `DatabaseMetaData`
+  * Fix `RelToSqlConverterTest` on Windows
+* Following
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-897">CALCITE-897</a>],
+  empty string for `boolean` properties means true
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-992">CALCITE-992</a>]
+  Validate and resolve sequence reference as a `Table` object
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-968">CALCITE-968</a>]
+  Stream-to-relation and stream-to-stream joins (Milinda Pathirage)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1041">CALCITE-1041</a>]
+  User-defined function that returns `DATE` or `TIMESTAMP` value
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-986">CALCITE-986</a>]
+  User-defined function with `DATE` or `TIMESTAMP` parameters
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-958">CALCITE-958</a>]
+  Overloaded Table Functions with named arguments (Julien Le Dem)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-970">CALCITE-970</a>]
+  If `NULLS FIRST`/`NULLS LAST` not specified, sort `NULL` values high
+
+Avatica features and bug fixes
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1040">CALCITE-1040</a>]
+  Differentiate better between arrays and scalars in protobuf
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-934">CALCITE-934</a>]
+  Use an OS-assigned ephemeral port for `CalciteRemoteDriverTest`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-767">CALCITE-767</a>]
+  Create Avatica RPC endpoints for commit and rollback commands
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-983">CALCITE-983</a>]
+  Handle nulls in `ErrorResponse`'s protobuf representation better
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-989">CALCITE-989</a>]
+  Add server's address in each response
+* Fix some bugs found by static analysis
+* Make all `equals` and `hashCode` methods uniform
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-962">CALCITE-962</a>]
+  Propagate the cause, not just the cause's message, from `JdbcMeta`
+
+Planner rules
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1057">CALCITE-1057</a>]
+  Add `RelMetadataProvider` parameter to standard planner `Program`s
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1055">CALCITE-1055</a>]
+  `SubQueryRemoveRule` should create `Correlate`, not `Join`, for correlated
+  sub-queries
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-978">CALCITE-978</a>]
+  Enable customizing constant folding rule behavior when a `Filter` simplifies
+  to false (Jason Altekruse)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-977">CALCITE-977</a>]
+  Make the constant expression `Executor` configurable in `FrameworkConfig`
+  (Jason Altekruse)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1058">CALCITE-1058</a>]
+  Add method `RelBuilder.empty`, and rewrite LIMIT 0 and WHERE FALSE to it
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-996">CALCITE-996</a>]
+  Simplify predicate when we create a `Filter` operator
+* Simplify `RexProgram`, in particular `(NOT CASE ... END) IS TRUE`, which
+  occurs in when `NOT IN` is expanded
+* Fix variant of
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-923">CALCITE-923</a>]
+  that occurs in `RelOptRulesTest.testPushFilterPastProject`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1023">CALCITE-1023</a>]
+  and
+  [<a href="https://issues.apache.org/jira/browse/CALCITE-1038">CALCITE-1038</a>]
+  Planner rule that removes `Aggregate` keys that are constant
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1018">CALCITE-1018</a>]
+  `SortJoinTransposeRule` not firing due to `getMaxRowCount(RelSubset)` returning
+  null
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1019">CALCITE-1019</a>]
+  `RelMdUtil.checkInputForCollationAndLimit()` was wrong with `alreadySorted`
+  check
+* Not safe to use '=' for predicates on constant expressions that might be null
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-993">CALCITE-993</a>]
+  Pull up all constant expressions, not just literals, as predicates
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1005">CALCITE-1005</a>]
+  Handle null in `getMaxRowCount` for `Aggregate` (Mike Hinchey)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-995">CALCITE-995</a>]
+  Sort transpose rules might fall in an infinite loop
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-987">CALCITE-987</a>]
+  Pushing `LIMIT 0` results in an infinite loop (Pengcheng Xiong)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-988">CALCITE-988</a>]
+  `FilterToProjectUnifyRule.invert(MutableRel, MutableRel, MutableProject)`
+  works incorrectly
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-969">CALCITE-969</a>]
+  Composite `EnumerableSort` with `DESC` wrongly sorts `NULL` values low
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-959">CALCITE-959</a>]
+  Add description to `SortProjectTransposeRule`'s constructor
+
+Bug fixes, API changes and minor enhancements
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1060">CALCITE-1060</a>]
+  Fix test deadlock by initializing `DriverManager` before registering `AlternatingDriver`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1047">CALCITE-1047</a>]
+  `ChunkList.clear` throws `AssertionError`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1053">CALCITE-1053</a>]
+  CPU spin, `ReflectiveRelMetadataProvider.apply` waiting for `HashMap.get`
+* Upgrade toolbox, to fix line length issue on Windows
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1051">CALCITE-1051</a>]
+  Underflow exception due to scaling IN clause literals (Frankie Bollaert)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-975">CALCITE-975</a>]
+  Allow Planner to return validated row type together with SqlNode
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1020">CALCITE-1020</a>]
+  Add `MILLISECOND` in `TimeUnit` (Pengcheng Xiong)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-794">CALCITE-794</a>]
+  Detect cycles when computing statistics
+  (**This is a breaking change**.)
+* Tune algorithm that deduces the return type of `AND` expression
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-842">CALCITE-842</a>]
+  Decorrelator gets field offsets confused if fields have been trimmed
+* Fix `NullPointerException` in `SqlJoin.toString()`
+* Add `ImmutableBitSet.rebuild()`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-915">CALCITE-915</a>]
+  Tests now unset `ThreadLocal` values on exit
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1036">CALCITE-1036</a>]
+  `DiffRepository` should not insert new resources at the end of the repository
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-955">CALCITE-955</a>]
+  `Litmus` (continuation-passing style for methods that check invariants)
+* `RelBuilder.project` now does nothing if asked to project the identity with
+  the same field names
+* Deprecate some `Util` methods, and upgrade last Maven modules to JDK 1.7
+* Document `RelOptPredicateList`
+* Add `ImmutableNullableList.copyOf(Iterable)`
+* Fix "endPosTable already set" error from `javac`
+* Add benchmark of `Parser.create(sql).parseQuery()`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1042">CALCITE-1042</a>]
+  Ensure that `FILTER` is `BOOLEAN NOT NULL`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1039">CALCITE-1039</a>]
+  Assign a `SqlKind` value for each built-in aggregate function
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1030">CALCITE-1030</a>]
+  JSON `ModelHandler` calling `SchemaPlus.setCacheEnabled()` causes
+  `UnsupportedOperationException` when using `SimpleCalciteSchema`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1028">CALCITE-1028</a>]
+  Move populate materializations after sql-to-rel conversion
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1034">CALCITE-1034</a>]
+  Use a custom checker for code style rules that Checkstyle cannot express
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1032">CALCITE-1032</a>]
+  Verify javadoc of private methods
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1015">CALCITE-1015</a>]
+  `OFFSET 0` causes `AssertionError` (Zhen Wang)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1024">CALCITE-1024</a>]
+  In a planner test, if a rule should have no effect, state that explicitly
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1016">CALCITE-1016</a>]
+  `GROUP BY *constant*` on empty relation should return 0 rows
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1022">CALCITE-1022</a>]
+  Rename `.oq` Quidem files to `.iq`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-980">CALCITE-980</a>]
+  Fix `AND` and `OR` implementation in `Enumerable` convention
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-459">CALCITE-459</a>]
+  When parsing SQL, allow single line comment on last line (Zhen Wang)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1009">CALCITE-1009</a>]
+  `SelfPopulatingList` is not thread-safe
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1008">CALCITE-1008</a>]
+  Replace `Closeable` with `AutoCloseable`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1001">CALCITE-1001</a>]
+  Upgrade to quidem-0.7
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-990">CALCITE-990</a>]
+  In `VolcanoPlanner`, populate `RelOptRuleCall.nodeInputs` for operands of type
+  "any"
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-966">CALCITE-966</a>]
+  `VolcanoPlanner` now clears `ruleNames` in order to avoid rule name
+  conflicting error
+* Factor user-defined function tests from `JdbcTest` to `UdfTest`, and classes
+  into `Smalls`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-974">CALCITE-974</a>]
+  Exception while validating `DELETE` (Yuri Au Yong)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-964">CALCITE-964</a>]
+  Rename `timezone` connection property to `timeZone`
+
+Web site and documentation
+
+* Avatica
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-1033">CALCITE-1033</a>]
+    Introduce Avatica protobuf documentation
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-1029">CALCITE-1029</a>]
+     Add "purpose" descriptions to Avatica JSON docs
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-984">CALCITE-984</a>]
+    Massive cleanup of Avatica JSON docs
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-861">CALCITE-861</a>]
+  Be explicit that `mvn test` needs to be invoked
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-997">CALCITE-997</a>]
+  Document keywords
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-979">CALCITE-979</a>]
+  Broken links in web site
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-961">CALCITE-961</a>]
+  Web site: Add downloads and Apache navigation links
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-960">CALCITE-960</a>]
+  Download links for pgp, md5, `KEYS` files, and direct from mirrors
+* Remove embedded date-stamps from javadoc; add javadoc for test classes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-965">CALCITE-965</a>]
+  Link to downloads page from each release news item
+
 ## <a href="https://github.com/apache/calcite/releases/tag/calcite-1.5.0">1.5.0</a> / 2015-11-06
 {: #v1-5-0}
 
@@ -201,6 +433,8 @@ RelBuilder and Piglet
 * Multisets and `COLLECT` in Piglet
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-785">CALCITE-785</a>]
   Add "Piglet", a subset of Pig Latin on top of Calcite algebra
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-869">CALCITE-869</a>]
+  Add `VALUES` command to Piglet
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-868">CALCITE-868</a>]
   Add API to execute queries expressed as `RelNode`
 * In RelBuilder, build expressions by table alias

http://git-wip-us.apache.org/repos/asf/calcite/blob/df5ec6dd/site/_docs/howto.md
----------------------------------------------------------------------
diff --git a/site/_docs/howto.md b/site/_docs/howto.md
index 2f2bddc..5389147 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.5.0-source.tar.gz
-$ cd calcite-1.5.0
+$ tar xvfz calcite-1.6.0-source.tar.gz
+$ cd calcite-1.6.0
 $ mvn install
 {% endhighlight %}
 
@@ -414,7 +414,7 @@ Before you start:
 
 * Set up signing keys as described above.
 * Make sure you are using JDK 1.7 (not 1.8).
-* Check that `README`, `README.md` and `doc/howto.md` have the correct version number.
+* Check that `README` and `site/_docs/howto.md` have the correct version number.
 * Set `version.major` and `version.minor` in `pom.xml`.
 * Make sure build and tests succeed, including with
   -Dcalcite.test.db={mysql,hsqldb}, -Dcalcite.test.slow,


[09/50] [abbrv] calcite git commit: Add ImmutableNullableList.copyOf(Iterable)

Posted by jh...@apache.org.
Add ImmutableNullableList.copyOf(Iterable)


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

Branch: refs/heads/branch-release
Commit: b1fdd1280aafc883d7e088d5527621a789f6170d
Parents: ba1eee1
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jul 28 17:13:27 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:23 2016 -0800

----------------------------------------------------------------------
 .../calcite/util/ImmutableNullableList.java     | 28 ++++++++++++++++++++
 .../java/org/apache/calcite/util/UtilTest.java  | 27 ++++++++++++++++++-
 2 files changed, 54 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/b1fdd128/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java b/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java
index 8ba9a6e..202ee5e 100644
--- a/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java
+++ b/core/src/main/java/org/apache/calcite/util/ImmutableNullableList.java
@@ -22,6 +22,7 @@ import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 
 import java.util.AbstractList;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
@@ -74,6 +75,33 @@ public class ImmutableNullableList<E> extends AbstractList<E> {
    * Returns an immutable list containing the given elements, in order.
    *
    * <p>Behavior as
+   * {@link com.google.common.collect.ImmutableList#copyOf(Iterable)}
+   * except that this list allows nulls.
+   */
+  public static <E> List<E> copyOf(Iterable<? extends E> elements) {
+    if (elements instanceof ImmutableNullableList
+        || elements instanceof ImmutableList
+        || elements == SINGLETON_NULL) {
+      //noinspection unchecked
+      return (List<E>) elements;
+    }
+    if (elements instanceof Collection) {
+      //noinspection unchecked
+      return copyOf((Collection) elements);
+    }
+    // If there are no nulls, ImmutableList is better.
+    final List<E> list = new ArrayList<>();
+    Iterables.addAll(list, elements);
+    if (list.contains(null)) {
+      return ImmutableNullableList.copyOf(list);
+    }
+    return ImmutableList.copyOf(elements);
+  }
+
+  /**
+   * Returns an immutable list containing the given elements, in order.
+   *
+   * <p>Behavior as
    * {@link com.google.common.collect.ImmutableList#copyOf(Object[])}
    * except that this list allows nulls.</p>
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/b1fdd128/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 9a42fcc..78202e2 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -72,6 +72,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.isA;
+import static org.hamcrest.CoreMatchers.not;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
@@ -1282,8 +1283,32 @@ public class UtilTest {
         isA((Class) ImmutableList.class));
 
     // list with no nulls uses ImmutableList
-    assertThat(ImmutableNullableList.copyOf(Arrays.asList("a", "b", "c")),
+    final List<String> abcList = Arrays.asList("a", "b", "c");
+    assertThat(ImmutableNullableList.copyOf(abcList),
         isA((Class) ImmutableList.class));
+
+    // list with no nulls uses ImmutableList
+    final Iterable<String> abc =
+        new Iterable<String>() {
+          public Iterator<String> iterator() {
+            return abcList.iterator();
+          }
+        };
+    assertThat(ImmutableNullableList.copyOf(abc),
+        isA((Class) ImmutableList.class));
+    assertThat(ImmutableNullableList.copyOf(abc), equalTo(abcList));
+
+    // list with no nulls uses ImmutableList
+    final List<String> ab0cList = Arrays.asList("a", "b", null, "c");
+    final Iterable<String> ab0c =
+        new Iterable<String>() {
+          public Iterator<String> iterator() {
+            return ab0cList.iterator();
+          }
+        };
+    assertThat(ImmutableNullableList.copyOf(ab0c),
+        not(isA((Class) ImmutableList.class)));
+    assertThat(ImmutableNullableList.copyOf(ab0c), equalTo(ab0cList));
   }
 
   /** Test for {@link org.apache.calcite.util.UnmodifiableArrayList}. */


[05/50] [abbrv] calcite git commit: [CALCITE-1038] Obsolete AggregateConstantKeyRule

Posted by jh...@apache.org.
[CALCITE-1038] Obsolete AggregateConstantKeyRule

Move AggregateConstantKeyRule's functionality into
AggregateProjectPullUpConstantsRule and obsolete it.

Detect constants using metadata (RelMdPredicates). The input does not
need to be a Project, and constants do not need to be literals.


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

Branch: refs/heads/branch-release
Commit: 904c73da60b9f9deec61ea34d89ada3462381f93
Parents: 8531ab1
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 4 10:35:43 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     |   4 +-
 .../rel/rules/AggregateConstantKeyRule.java     | 130 ---------------
 .../AggregateProjectPullUpConstantsRule.java    | 163 +++++++++----------
 .../rel/rules/ReduceExpressionsRule.java        |  67 +++++---
 .../apache/calcite/test/RelOptRulesTest.java    |  15 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml |  14 +-
 6 files changed, 135 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/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 e658749..e444612 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -39,7 +39,6 @@ import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.rules.AggregateConstantKeyRule;
 import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
 import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.calcite.rel.rules.MultiJoin;
@@ -1546,8 +1545,7 @@ public abstract class RelOptUtil {
   }
 
   public static void registerAbstractRels(RelOptPlanner planner) {
-    planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE);
-    planner.addRule(AggregateConstantKeyRule.INSTANCE);
+    planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE2);
     planner.addRule(PruneEmptyRules.UNION_INSTANCE);
     planner.addRule(PruneEmptyRules.PROJECT_INSTANCE);
     planner.addRule(PruneEmptyRules.FILTER_INSTANCE);

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
deleted file mode 100644
index ea31178..0000000
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
+++ /dev/null
@@ -1,130 +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.rel.rules;
-
-import org.apache.calcite.plan.RelOptPredicateList;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.ImmutableBitSet;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-/**
- * Planner rule that removes constant keys from an
- * a {@link Aggregate}.
- *
- * <p>It never removes the last column, because {@code Aggregate([])} returns
- * 1 row even if its input is empty.
- */
-public class AggregateConstantKeyRule extends RelOptRule {
-  public static final AggregateConstantKeyRule INSTANCE =
-      new AggregateConstantKeyRule(RelFactories.LOGICAL_BUILDER,
-          "AggregateConstantKeyRule");
-
-  //~ Constructors -----------------------------------------------------------
-
-  /** Creates an AggregateConstantKeyRule. */
-  private AggregateConstantKeyRule(RelBuilderFactory relBuilderFactory,
-      String description) {
-    super(operand(Aggregate.class, null, Aggregate.IS_SIMPLE, any()),
-        relBuilderFactory, description);
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  public void onMatch(RelOptRuleCall call) {
-    final Aggregate aggregate = call.rel(0);
-    assert !aggregate.indicator : "predicate ensured no grouping sets";
-
-    final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
-    final RelOptPredicateList predicates =
-        RelMetadataQuery.getPulledUpPredicates(aggregate.getInput());
-    if (predicates == null) {
-      return;
-    }
-    final ImmutableMap<RexNode, RexLiteral> constants =
-        ReduceExpressionsRule.predicateConstants(rexBuilder, predicates);
-    final NavigableMap<Integer, RexLiteral> map = new TreeMap<>();
-    for (int key : aggregate.getGroupSet()) {
-      final RexInputRef ref =
-          rexBuilder.makeInputRef(aggregate.getInput(), key);
-      if (constants.containsKey(ref)) {
-        map.put(key, constants.get(ref));
-      }
-    }
-
-    if (map.isEmpty()) {
-      return; // none of the keys are constant
-    }
-
-    if (map.size() == aggregate.getGroupCount()) {
-      if (map.size() == 1) {
-        // There is one key, and it is constant. We cannot remove it.
-        return;
-      }
-      map.remove(map.descendingKeySet().descendingIterator().next());
-    }
-
-    ImmutableBitSet newGroupSet = aggregate.getGroupSet();
-    for (int key : map.keySet()) {
-      newGroupSet = newGroupSet.clear(key);
-    }
-    final Aggregate newAggregate =
-        aggregate.copy(aggregate.getTraitSet(), aggregate.getInput(),
-            false, newGroupSet, ImmutableList.of(newGroupSet),
-            aggregate.getAggCallList());
-    final RelBuilder relBuilder = call.builder();
-    relBuilder.push(newAggregate);
-
-    final List<RexNode> projects = new ArrayList<>();
-    int offset = 0;
-    for (RelDataTypeField field : aggregate.getRowType().getFieldList()) {
-      RexNode node = null;
-      if (field.getIndex() < aggregate.getGroupCount()) {
-        node = map.get(aggregate.getGroupSet().nth(field.getIndex()));
-        if (node != null) {
-          node = relBuilder.getRexBuilder().makeCast(field.getType(), node, true);
-          node = relBuilder.alias(node, field.getName());
-          ++offset;
-        }
-      }
-      if (node == null) {
-        node = relBuilder.field(field.getIndex() - offset);
-      }
-      projects.add(node);
-    }
-    call.transformTo(relBuilder.project(projects).build());
-  }
-}
-
-// End AggregateConstantKeyRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/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 2b99cf6..a0ff130 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
@@ -16,70 +16,91 @@
  */
 package org.apache.calcite.rel.rules;
 
+import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
-import org.apache.calcite.util.mapping.Mapping;
-import org.apache.calcite.util.mapping.MappingType;
+
+import com.google.common.collect.ImmutableMap;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 /**
- * Planner rule that removes constant expressions from the
- * group list of an {@link org.apache.calcite.rel.logical.LogicalAggregate}.
+ * Planner rule that removes constant keys from an
+ * {@link org.apache.calcite.rel.core.Aggregate}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
  *
- * <p><b>Effect of the rule</b></p>
+ * <p>This rules never removes the last column, because {@code Aggregate([])}
+ * returns 1 row even if its input is empty.
  *
  * <p>Since the transformed relational expression has to match the original
  * relational expression, the constants are placed in a projection above the
  * reduced aggregate. If those constants are not used, another rule will remove
  * them from the project.
- *
- * <p>LogicalAggregate needs its group columns to be on the prefix of its input
- * relational expression. Therefore, if a constant is not on the trailing edge
- * of the group list, removing it will leave a hole. In this case, the rule adds
- * a project before the aggregate to reorder the columns, and permutes them back
- * afterwards.
  */
 public class AggregateProjectPullUpConstantsRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
   public static final AggregateProjectPullUpConstantsRule INSTANCE =
-      new AggregateProjectPullUpConstantsRule();
+      new AggregateProjectPullUpConstantsRule(LogicalAggregate.class,
+          LogicalProject.class, RelFactories.LOGICAL_BUILDER,
+          "AggregateProjectPullUpConstantsRule");
+
+  /** More general instance that matches any relational expression. */
+  public static final AggregateProjectPullUpConstantsRule INSTANCE2 =
+      new AggregateProjectPullUpConstantsRule(LogicalAggregate.class,
+          RelNode.class, RelFactories.LOGICAL_BUILDER,
+          "AggregatePullUpConstantsRule");
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Private: use singleton
+   * Creates an AggregateProjectPullUpConstantsRule.
+   *
+   * @param aggregateClass Aggregate class
+   * @param inputClass Input class, such as {@link LogicalProject}
+   * @param relBuilderFactory Builder for relational expressions
+   * @param description Description, or null to guess description
    */
-  private AggregateProjectPullUpConstantsRule() {
+  public AggregateProjectPullUpConstantsRule(
+      Class<? extends Aggregate> aggregateClass,
+      Class<? extends RelNode> inputClass,
+      RelBuilderFactory relBuilderFactory, String description) {
     super(
-        operand(LogicalAggregate.class, null, Aggregate.IS_SIMPLE,
-            operand(LogicalProject.class, any())));
+        operand(aggregateClass, null, Aggregate.IS_SIMPLE,
+            operand(inputClass, any())),
+        relBuilderFactory, description);
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final LogicalAggregate aggregate = call.rel(0);
-    final LogicalProject input = call.rel(1);
+    final Aggregate aggregate = call.rel(0);
+    final RelNode input = call.rel(1);
 
+    assert !aggregate.indicator : "predicate ensured no grouping sets";
     final int groupCount = aggregate.getGroupCount();
     if (groupCount == 1) {
       // No room for optimization since we cannot convert from non-empty
@@ -87,47 +108,49 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       return;
     }
 
-    final RexProgram program =
-      RexProgram.create(input.getInput().getRowType(),
-          input.getProjects(),
-          null,
-          input.getRowType(),
-          input.getCluster().getRexBuilder());
-
-    final RelDataType childRowType = input.getRowType();
-    final List<Integer> constantList = new ArrayList<>();
-    final Map<Integer, RexNode> constants = new HashMap<>();
-    for (int i : aggregate.getGroupSet()) {
-      final RexLocalRef ref = program.getProjectList().get(i);
-      if (program.isConstant(ref)) {
-        constantList.add(i);
-        constants.put(
-            i,
-            program.gatherExpr(ref));
+    final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
+    final RelOptPredicateList predicates =
+        RelMetadataQuery.getPulledUpPredicates(aggregate.getInput());
+    if (predicates == null) {
+      return;
+    }
+    final ImmutableMap<RexNode, RexNode> constants =
+        ReduceExpressionsRule.predicateConstants(RexNode.class, rexBuilder,
+            predicates);
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int key : aggregate.getGroupSet()) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(aggregate.getInput(), key);
+      if (constants.containsKey(ref)) {
+        map.put(key, constants.get(ref));
       }
     }
 
     // None of the group expressions are constant. Nothing to do.
-    if (constantList.size() == 0) {
+    if (map.isEmpty()) {
       return;
     }
 
-    if (groupCount == constantList.size()) {
+    if (groupCount == map.size()) {
       // At least a single item in group by is required.
-      // Otherwise group by 1,2 might be altered to group by ()
+      // Otherwise "GROUP BY 1, 2" might be altered to "GROUP BY ()".
       // Removing of the first element is not optimal here,
       // however it will allow us to use fast path below (just trim
-      // groupCount)
-      constantList.remove(0);
+      // groupCount).
+      map.remove(map.navigableKeySet().first());
     }
 
-    final int newGroupCount = groupCount - constantList.size();
+    ImmutableBitSet newGroupSet = aggregate.getGroupSet();
+    for (int key : map.keySet()) {
+      newGroupSet = newGroupSet.clear(key);
+    }
+    final int newGroupCount = newGroupSet.cardinality();
 
     // If the constants are on the trailing edge of the group list, we just
     // reduce the group count.
     final RelBuilder relBuilder = call.builder();
     relBuilder.push(input);
-    if (constantList.get(0) == newGroupCount) {
+    if (map.navigableKeySet().first() == newGroupCount) {
       // Clone aggregate calls.
       final List<AggregateCall> newAggCalls = new ArrayList<>();
       for (AggregateCall aggCall : aggregate.getAggCallList()) {
@@ -136,13 +159,13 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
                 groupCount, newGroupCount));
       }
       relBuilder.aggregate(
-          relBuilder.groupKey(ImmutableBitSet.range(newGroupCount), false, null),
+          relBuilder.groupKey(newGroupSet, false, null),
           newAggCalls);
     } else {
       // Create the mapping from old field positions to new field
       // positions.
       final Permutation mapping =
-          new Permutation(childRowType.getFieldCount());
+          new Permutation(input.getRowType().getFieldCount());
       mapping.identity();
 
       // Ensure that the first positions in the mapping are for the new
@@ -150,18 +173,13 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       for (int i = 0, groupOrdinal = 0, constOrdinal = newGroupCount;
           i < groupCount;
           ++i) {
-        if (i >= groupCount) {
-          mapping.set(i, i);
-        } else if (constants.containsKey(i)) {
+        if (map.containsKey(i)) {
           mapping.set(i, constOrdinal++);
         } else {
           mapping.set(i, groupOrdinal++);
         }
       }
 
-      // Create a projection to permute fields into these positions.
-      createProjection(relBuilder, mapping);
-
       // Adjust aggregate calls for new field positions.
       final List<AggregateCall> newAggCalls = new ArrayList<>();
       for (AggregateCall aggCall : aggregate.getAggCallList()) {
@@ -180,7 +198,7 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
 
       // Aggregate on projection.
       relBuilder.aggregate(
-          relBuilder.groupKey(ImmutableBitSet.range(newGroupCount), false, null),
+          relBuilder.groupKey(newGroupSet, false, null),
               newAggCalls);
     }
 
@@ -192,10 +210,10 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       final int i = field.getIndex();
       if (i >= groupCount) {
         // Aggregate expressions' names and positions are unchanged.
-        expr = relBuilder.field(i - constantList.size());
-      } else if (constantList.contains(i)) {
+        expr = relBuilder.field(i - map.size());
+      } else if (map.containsKey(i)) {
         // Re-generate the constant expression in the project.
-        expr = constants.get(i);
+        expr = map.get(i);
       } else {
         // Project the aggregation expression, in its original
         // position.
@@ -208,33 +226,6 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
     call.transformTo(relBuilder.build());
   }
 
-  /**
-   * Creates a projection which permutes the fields of a given relational
-   * expression.
-   *
-   * <p>For example, given a relational expression [A, B, C, D] and a mapping
-   * [2:1, 3:0], returns a projection [$3 AS C, $2 AS B].
-   *
-   * @param relBuilder Relational expression builder
-   * @param mapping Mapping to apply to source columns
-   */
-  private static RelBuilder createProjection(RelBuilder relBuilder,
-      Mapping mapping) {
-    // Every target has precisely one source; every source has at most
-    // one target.
-    assert mapping.getMappingType().isA(MappingType.INVERSE_SURJECTION);
-    final RelDataType childRowType = relBuilder.peek().getRowType();
-    assert mapping.getSourceCount() == childRowType.getFieldCount();
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
-    for (int target = 0; target < mapping.getTargetCount(); ++target) {
-      int source = mapping.getSource(target);
-      projects.add(
-          Pair.<RexNode, String>of(
-              relBuilder.field(source),
-              childRowType.getFieldList().get(source).getName()));
-    }
-    return relBuilder.project(Pair.left(projects), Pair.right(projects));
-  }
 }
 
 // End AggregateProjectPullUpConstantsRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 8be202e..17d5cb0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -396,7 +396,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     List<Boolean> addCasts = Lists.newArrayList();
     final List<RexNode> removableCasts = Lists.newArrayList();
     final ImmutableMap<RexNode, RexLiteral> constants =
-        predicateConstants(rexBuilder, predicates);
+        predicateConstants(RexLiteral.class, rexBuilder, predicates);
     findReducibleExps(rel.getCluster().getTypeFactory(), expList, constants,
         constExps, addCasts, removableCasts);
     if (constExps.isEmpty() && removableCasts.isEmpty()) {
@@ -495,7 +495,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
    * @param removableCasts returns the list of cast expressions where the cast
    */
   protected static void findReducibleExps(RelDataTypeFactory typeFactory,
-      List<RexNode> exps, ImmutableMap<RexNode, RexLiteral> constants,
+      List<RexNode> exps, ImmutableMap<RexNode, ? extends RexNode> constants,
       List<RexNode> constExps, List<Boolean> addCasts,
       List<RexNode> removableCasts) {
     ReducibleExprLocator gardener =
@@ -507,7 +507,19 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     assert constExps.size() == addCasts.size();
   }
 
-  protected static ImmutableMap<RexNode, RexLiteral> predicateConstants(
+  /** Creates a map containing each (e, constant) pair that occurs within
+   * a predicate list.
+   *
+   * @param clazz Class of expression that is considered constant
+   * @param rexBuilder Rex builder
+   * @param predicates Predicate list
+   * @param <C> what to consider a constant: {@link RexLiteral} to use a narrow
+   *           definition of constant, or {@link RexNode} to use
+   *           {@link RexUtil#isConstant(RexNode)}
+   * @return Map from values to constants
+   */
+  protected static <C extends RexNode> ImmutableMap<RexNode, C>
+  predicateConstants(Class<C> clazz,
       RexBuilder rexBuilder, RelOptPredicateList predicates) {
     // We cannot use an ImmutableMap.Builder here. If there are multiple entries
     // with the same key (e.g. "WHERE deptno = 1 AND deptno = 2"), it doesn't
@@ -518,14 +530,14 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     // (e.g. "WHERE deptno = 1 AND deptno = 1")
     // (3) It will return false if there are inconsistent constraints (e.g.
     // "WHERE deptno = 1 AND deptno = 2")
-    final Map<RexNode, RexLiteral> map = new HashMap<>();
+    final Map<RexNode, C> map = new HashMap<>();
     final Set<RexNode> excludeSet = new HashSet<>();
     for (RexNode predicate : predicates.pulledUpPredicates) {
-      gatherConstraints(predicate, map, excludeSet, rexBuilder);
+      gatherConstraints(clazz, predicate, map, excludeSet, rexBuilder);
     }
-    final ImmutableMap.Builder<RexNode, RexLiteral> builder =
+    final ImmutableMap.Builder<RexNode, C> builder =
         ImmutableMap.builder();
-    for (Map.Entry<RexNode, RexLiteral> entry : map.entrySet()) {
+    for (Map.Entry<RexNode, C> entry : map.entrySet()) {
       RexNode rexNode = entry.getKey();
       if (!overlap(rexNode, excludeSet)) {
         builder.put(rexNode, entry.getValue());
@@ -559,8 +571,8 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     }
   }
 
-  private static void gatherConstraints(RexNode predicate,
-      Map<RexNode, RexLiteral> map, Set<RexNode> excludeSet,
+  private static <C extends RexNode> void gatherConstraints(Class<C> clazz,
+      RexNode predicate, Map<RexNode, C> map, Set<RexNode> excludeSet,
       RexBuilder rexBuilder) {
     if (predicate.getKind() != SqlKind.EQUALS) {
       decompose(excludeSet, predicate);
@@ -576,21 +588,24 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     final RexNode right = operands.get(1);
     // note that literals are immutable too and they can only be compared through
     // values.
-    if (right instanceof RexLiteral) {
-      foo(left, (RexLiteral) right, map, excludeSet, rexBuilder);
-    }
-    if (left instanceof RexLiteral) {
-      foo(right, (RexLiteral) left, map, excludeSet, rexBuilder);
-    }
+    gatherConstraint(clazz, left, right, map, excludeSet, rexBuilder);
+    gatherConstraint(clazz, right, left, map, excludeSet, rexBuilder);
   }
 
-  private static void foo(RexNode left, RexLiteral right,
-      Map<RexNode, RexLiteral> map, Set<RexNode> excludeSet,
+  private static <C extends RexNode> void gatherConstraint(Class<C> clazz,
+      RexNode left, RexNode right, Map<RexNode, C> map, Set<RexNode> excludeSet,
       RexBuilder rexBuilder) {
+    if (!clazz.isInstance(right)) {
+      return;
+    }
+    if (!RexUtil.isConstant(right)) {
+      return;
+    }
+    C constant = clazz.cast(right);
     if (excludeSet.contains(left)) {
       return;
     }
-    final RexLiteral existedValue = map.get(left);
+    final C existedValue = map.get(left);
     if (existedValue == null) {
       switch (left.getKind()) {
       case CAST:
@@ -599,16 +614,19 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
         final RexNode operand = ((RexCall) left).getOperands().get(0);
         if (canAssignFrom(left.getType(), operand.getType())) {
           final RexNode castRight =
-              rexBuilder.makeCast(operand.getType(), right);
+              rexBuilder.makeCast(operand.getType(), constant);
           if (castRight instanceof RexLiteral) {
             left = operand;
-            right = (RexLiteral) castRight;
+            constant = clazz.cast(castRight);
           }
         }
       }
-      map.put(left, right);
+      map.put(left, constant);
     } else {
-      if (!existedValue.getValue().equals(right.getValue())) {
+      if (existedValue instanceof RexLiteral
+          && constant instanceof RexLiteral
+          && !((RexLiteral) existedValue).getValue()
+              .equals(((RexLiteral) constant).getValue())) {
         // we found conflicting values, e.g. left = 10 and left = 20
         map.remove(left);
         excludeSet.add(left);
@@ -770,7 +788,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
 
     private final List<Constancy> stack;
 
-    private final ImmutableMap<RexNode, RexLiteral> constants;
+    private final ImmutableMap<RexNode, ? extends RexNode> constants;
 
     private final List<RexNode> constExprs;
 
@@ -781,7 +799,8 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     private final List<SqlOperator> parentCallTypeStack;
 
     ReducibleExprLocator(RelDataTypeFactory typeFactory,
-        ImmutableMap<RexNode, RexLiteral> constants, List<RexNode> constExprs,
+        ImmutableMap<RexNode, ? extends RexNode> constants,
+        List<RexNode> constExprs,
         List<Boolean> addCasts, List<RexNode> removableCasts) {
       // go deep
       super(true);

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/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 4061b0d..92bae3e 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -36,7 +36,6 @@ import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
-import org.apache.calcite.rel.rules.AggregateConstantKeyRule;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateFilterTransposeRule;
 import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
@@ -2027,7 +2026,7 @@ public class RelOptRulesTest extends RelOptTestBase {
    * Planner rule that removes Aggregate keys that are constant</a>. */
   @Test public void testAggregateConstantKeyRule() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select count(*) as c\n"
         + "from sales.emp\n"
@@ -2036,11 +2035,11 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanning(new HepPlanner(program), sql);
   }
 
-  /** Tests {@link AggregateConstantKeyRule} where reduction is not possible
-   * because "deptno" is the only key. */
+  /** Tests {@link AggregateProjectPullUpConstantsRule} where reduction is not
+   * possible because "deptno" is the only key. */
   @Test public void testAggregateConstantKeyRule2() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select count(*) as c\n"
         + "from sales.emp\n"
@@ -2049,11 +2048,11 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanUnchanged(new HepPlanner(program), sql);
   }
 
-  /** Tests {@link AggregateConstantKeyRule} where both keys are constants but
-   * only one can be removed. */
+  /** Tests {@link AggregateProjectPullUpConstantsRule} where both keys are
+   * constants but only one can be removed. */
   @Test public void testAggregateConstantKeyRule3() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select job\n"
         + "from sales.emp\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/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 38edf69..c9c9342 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -1795,8 +1795,8 @@ LogicalProject(DEPTNO=[$0], EXPR$1=[$4])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(DEPTNO=[$0], EXPR$1=[$2])
-  LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($4)])
-    LogicalProject(DEPTNO=[$7], DEPTNO42=[+($7, 42)], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], MGR=[$3])
+  LogicalAggregate(group=[{0, 3}], EXPR$1=[MAX($4)])
+    LogicalProject(DEPTNO=[$7], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)], MGR=[$3])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -1818,8 +1818,8 @@ LogicalProject(DEPTNO=[$1], EXPR$1=[$2])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
-  LogicalProject(DEPTNO=[$7], FOUR=[4], MGR=[$3])
+LogicalAggregate(group=[{1}], EXPR$1=[MAX($2)])
+  LogicalProject(FOUR=[4], DEPTNO=[$7], MGR=[$3])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -1841,8 +1841,8 @@ LogicalProject(DEPTNO=[$1], EXPR$1=[$2])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
-  LogicalProject(DEPTNO=[$7], $f0=[+(42, 24)], MGR=[$3])
+LogicalAggregate(group=[{1}], EXPR$1=[MAX($2)])
+  LogicalProject($f0=[+(42, 24)], DEPTNO=[$7], MGR=[$3])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -4311,7 +4311,7 @@ LogicalProject(JOB=[$1])
             <![CDATA[
 LogicalProject(JOB=[$1])
   LogicalFilter(condition=[>($2, 3)])
-    LogicalProject(SAL=[$0], JOB=['Clerk'], $f1=[$1])
+    LogicalProject(SAL=[$0], JOB=['Clerk'], $f2=[$1])
       LogicalAggregate(group=[{0}], agg#0=[COUNT()])
         LogicalProject(SAL=[$5], JOB=[$2])
           LogicalFilter(condition=[AND(IS NULL($5), =($2, 'Clerk'))])


[22/50] [abbrv] calcite git commit: [CALCITE-816] Represent sub-query as a RexNode

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
index 156a973..7f191a3 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
@@ -170,6 +170,36 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
     }
     SqlToRelTestBase.assertValid(relAfter);
   }
+
+  /** Sets the SQL statement for a test. */
+  Sql sql(String sql) {
+    return new Sql(sql, null, true);
+  }
+
+  /** Allows fluent testing. */
+  class Sql {
+    private final String sql;
+    private final HepPlanner hepPlanner;
+    private final boolean expand;
+
+    public Sql(String sql, HepPlanner hepPlanner, boolean expand) {
+      this.sql = sql;
+      this.hepPlanner = hepPlanner;
+      this.expand = expand;
+    }
+
+    public Sql with(HepPlanner hepPlanner) {
+      return new Sql(sql, hepPlanner, expand);
+    }
+
+    public Sql expand(boolean expand) {
+      return new Sql(sql, hepPlanner, expand);
+    }
+
+    public void check() {
+      checkPlanning(tester.withExpand(expand), null, hepPlanner, sql);
+    }
+  }
 }
 
 // End RelOptTestBase.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java b/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java
index 22bb852..0f88f0c 100644
--- a/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexTransformerTest.java
@@ -18,6 +18,7 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelOptUtil.Logic;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
@@ -25,6 +26,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.LogicVisitor;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
@@ -104,16 +106,10 @@ public class RexTransformerTest {
     if (null == encapsulateType) {
       root = node;
     } else if (encapsulateType.equals(Boolean.TRUE)) {
-      root =
-          rexBuilder.makeCall(
-              SqlStdOperatorTable.IS_TRUE,
-              node);
+      root = isTrue(node);
     } else {
       // encapsulateType.equals(Boolean.FALSE)
-      root =
-          rexBuilder.makeCall(
-              SqlStdOperatorTable.IS_FALSE,
-              node);
+      root = isFalse(node);
     }
 
     RexTransformer transformer = new RexTransformer(root, rexBuilder);
@@ -126,6 +122,59 @@ public class RexTransformerTest {
     }
   }
 
+  private RexNode lessThan(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, a0, a1);
+  }
+
+  private RexNode lessThanOrEqual(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, a0, a1);
+  }
+
+  private RexNode greaterThan(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, a0, a1);
+  }
+
+  private RexNode greaterThanOrEqual(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, a0,
+        a1);
+  }
+
+  private RexNode equals(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, a0, a1);
+  }
+
+  private RexNode notEquals(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, a0, a1);
+  }
+
+  private RexNode and(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.AND, a0, a1);
+  }
+
+  private RexNode or(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.OR, a0, a1);
+  }
+
+  private RexNode not(RexNode a0) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.NOT, a0);
+  }
+
+  private RexNode plus(RexNode a0, RexNode a1) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.PLUS, a0, a1);
+  }
+
+  private RexNode isNotNull(RexNode a0) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, a0);
+  }
+
+  private RexNode isFalse(RexNode node) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.IS_FALSE, node);
+  }
+
+  private RexNode isTrue(RexNode node) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, node);
+  }
+
   @Test public void testPreTests() {
     // can make variable nullable?
     RexNode node =
@@ -147,11 +196,7 @@ public class RexTransformerTest {
   }
 
   @Test public void testNonBooleans() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.PLUS,
-            x,
-            y);
+    RexNode node = plus(x, y);
     String expected = node.toString();
     check(Boolean.TRUE, node, expected);
     check(Boolean.FALSE, node, expected);
@@ -165,11 +210,7 @@ public class RexTransformerTest {
    * could be produced
    */
   @Test public void testOrUnchanged() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.OR,
-            x,
-            y);
+    RexNode node = or(x, y);
     String expected = node.toString();
     check(Boolean.TRUE, node, expected);
     check(Boolean.FALSE, node, expected);
@@ -177,11 +218,7 @@ public class RexTransformerTest {
   }
 
   @Test public void testSimpleAnd() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.AND,
-            x,
-            y);
+    RexNode node = and(x, y);
     check(
         Boolean.FALSE,
         node,
@@ -189,11 +226,7 @@ public class RexTransformerTest {
   }
 
   @Test public void testSimpleEquals() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            x,
-            y);
+    RexNode node = equals(x, y);
     check(
         Boolean.TRUE,
         node,
@@ -201,11 +234,7 @@ public class RexTransformerTest {
   }
 
   @Test public void testSimpleNotEquals() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.NOT_EQUALS,
-            x,
-            y);
+    RexNode node = notEquals(x, y);
     check(
         Boolean.FALSE,
         node,
@@ -213,11 +242,7 @@ public class RexTransformerTest {
   }
 
   @Test public void testSimpleGreaterThan() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN,
-            x,
-            y);
+    RexNode node = greaterThan(x, y);
     check(
         Boolean.TRUE,
         node,
@@ -225,11 +250,7 @@ public class RexTransformerTest {
   }
 
   @Test public void testSimpleGreaterEquals() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            x,
-            y);
+    RexNode node = greaterThanOrEqual(x, y);
     check(
         Boolean.FALSE,
         node,
@@ -237,11 +258,7 @@ public class RexTransformerTest {
   }
 
   @Test public void testSimpleLessThan() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN,
-            x,
-            y);
+    RexNode node = lessThan(x, y);
     check(
         Boolean.TRUE,
         node,
@@ -249,11 +266,7 @@ public class RexTransformerTest {
   }
 
   @Test public void testSimpleLessEqual() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            x,
-            y);
+    RexNode node = lessThanOrEqual(x, y);
     check(
         Boolean.FALSE,
         node,
@@ -261,17 +274,9 @@ public class RexTransformerTest {
   }
 
   @Test public void testOptimizeNonNullLiterals() {
-    RexNode node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            x,
-            trueRex);
+    RexNode node = lessThanOrEqual(x, trueRex);
     check(Boolean.TRUE, node, "AND(IS NOT NULL($0), <=($0, true))");
-    node =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-            trueRex,
-            x);
+    node = lessThanOrEqual(trueRex, x);
     check(Boolean.FALSE, node, "AND(IS NOT NULL($0), <=(true, $0))");
   }
 
@@ -282,16 +287,8 @@ public class RexTransformerTest {
 
   @Test public void testMixed1() {
     // x=true AND y
-    RexNode op1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            x,
-            trueRex);
-    RexNode and =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.AND,
-            op1,
-            y);
+    RexNode op1 = equals(x, trueRex);
+    RexNode and = and(op1, y);
     check(
         Boolean.FALSE,
         and,
@@ -300,21 +297,9 @@ public class RexTransformerTest {
 
   @Test public void testMixed2() {
     // x!=true AND y>z
-    RexNode op1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.NOT_EQUALS,
-            x,
-            trueRex);
-    RexNode op2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN,
-            y,
-            z);
-    RexNode and =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.AND,
-            op1,
-            op2);
+    RexNode op1 = notEquals(x, trueRex);
+    RexNode op2 = greaterThan(y, z);
+    RexNode and = and(op1, op2);
     check(
         Boolean.FALSE,
         and,
@@ -323,21 +308,9 @@ public class RexTransformerTest {
 
   @Test public void testMixed3() {
     // x=y AND false>z
-    RexNode op1 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.EQUALS,
-            x,
-            y);
-    RexNode op2 =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN,
-            falseRex,
-            z);
-    RexNode and =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.AND,
-            op1,
-            op2);
+    RexNode op1 = equals(x, y);
+    RexNode op2 = greaterThan(falseRex, z);
+    RexNode and = and(op1, op2);
     check(
         Boolean.TRUE,
         and,
@@ -396,6 +369,37 @@ public class RexTransformerTest {
     assertThat(leftJoinKeys.isEmpty(), is(true));
     assertThat(rightJoinKeys.isEmpty(), is(true));
   }
+
+  /** Test case for {@link org.apache.calcite.rex.LogicVisitor}. */
+  @Test public void testLogic() {
+    // x > FALSE AND ((y = z) IS NOT NULL)
+    final RexNode node = and(greaterThan(x, falseRex), isNotNull(equals(y, z)));
+    assertThat(deduceLogic(node, x, Logic.TRUE_FALSE),
+        is(Logic.TRUE_FALSE));
+    assertThat(deduceLogic(node, y, Logic.TRUE_FALSE),
+        is(Logic.TRUE_FALSE_UNKNOWN));
+    assertThat(deduceLogic(node, z, Logic.TRUE_FALSE),
+        is(Logic.TRUE_FALSE_UNKNOWN));
+
+    // TRUE means that a value of FALSE or UNKNOWN will kill the row
+    // (therefore we can safely use a semijoin)
+    assertThat(deduceLogic(and(x, y), x, Logic.TRUE), is(Logic.TRUE));
+    assertThat(deduceLogic(and(x, y), y, Logic.TRUE), is(Logic.TRUE));
+    assertThat(deduceLogic(and(x, and(y, z)), z, Logic.TRUE), is(Logic.TRUE));
+    assertThat(deduceLogic(and(x, not(y)), x, Logic.TRUE), is(Logic.TRUE));
+    assertThat(deduceLogic(and(x, not(y)), y, Logic.TRUE),
+        is(Logic.UNKNOWN_AS_TRUE));
+    assertThat(deduceLogic(and(x, not(and(y, z))), z, Logic.TRUE),
+        is(Logic.UNKNOWN_AS_TRUE));
+    assertThat(deduceLogic(or(x, y), x, Logic.TRUE), is(Logic.TRUE_FALSE));
+  }
+
+  private Logic deduceLogic(RexNode root, RexNode seek, Logic logic) {
+    final List<Logic> list = new ArrayList<>();
+    LogicVisitor.collect(root, seek, logic, list);
+    assertThat(list.size(), is(1));
+    return list.get(0);
+  }
 }
 
 // End RexTransformerTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 c24e6af..2e18dc7 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -138,6 +138,20 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
         "${plan}");
   }
 
+  @Test public void testJoinOnInSubQuery() {
+    final String sql = "select * from emp left join dept\n"
+        + "on emp.empno = 1\n"
+        + "or dept.deptno in (select deptno from emp where empno > 5)";
+    sql(sql).expand(false).convertsTo("${plan}");
+  }
+
+  @Test public void testJoinOnExists() {
+    final String sql = "select * from emp left join dept\n"
+        + "on emp.empno = 1\n"
+        + "or exists (select deptno from emp where empno > dept.deptno + 5)";
+    sql(sql).expand(false).convertsTo("${plan}");
+  }
+
   @Test public void testJoinUsing() {
     check("SELECT * FROM emp JOIN dept USING (deptno)", "${plan}");
   }
@@ -408,7 +422,8 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testDuplicateColumnsInSubQuery() {
     String sql = "select \"e\" from (\n"
         + "select empno as \"e\", deptno as d, 1 as \"e\" from EMP)";
-    tester.assertConvertsTo(sql, "${plan}");
+    sql(sql)
+        .convertsTo("${plan}");
   }
 
   @Test public void testOrder() {
@@ -629,27 +644,64 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testWithInsideWhereExists() {
-    tester.withDecorrelation(false).assertConvertsTo("select * from emp\n"
-            + "where exists (\n"
-            + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
-            + "  select 1 from dept2 where deptno <= emp.deptno)",
-        "${plan}");
+    final String sql = "select * from emp\n"
+        + "where exists (\n"
+        + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
+        + "  select 1 from dept2 where deptno <= emp.deptno)";
+    sql(sql)
+        .decorrelate(false)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testWithInsideWhereExistsRex() {
+    final String sql = "select * from emp\n"
+        + "where exists (\n"
+        + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
+        + "  select 1 from dept2 where deptno <= emp.deptno)";
+    sql(sql)
+        .decorrelate(false)
+        .expand(false)
+        .convertsTo("${plan}");
   }
 
   @Test public void testWithInsideWhereExistsDecorrelate() {
-    tester.withDecorrelation(true).assertConvertsTo("select * from emp\n"
-            + "where exists (\n"
-            + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
-            + "  select 1 from dept2 where deptno <= emp.deptno)",
-        "${plan}");
+    final String sql = "select * from emp\n"
+        + "where exists (\n"
+        + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
+        + "  select 1 from dept2 where deptno <= emp.deptno)";
+    sql(sql)
+        .decorrelate(true)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testWithInsideWhereExistsDecorrelateRex() {
+    final String sql = "select * from emp\n"
+        + "where exists (\n"
+        + "  with dept2 as (select * from dept where dept.deptno >= emp.deptno)\n"
+        + "  select 1 from dept2 where deptno <= emp.deptno)";
+    sql(sql)
+        .decorrelate(true)
+        .expand(false)
+        .convertsTo("${plan}");
   }
 
   @Test public void testWithInsideScalarSubquery() {
-    check("select (\n"
-            + " with dept2 as (select * from dept where deptno > 10)"
-            + " select count(*) from dept2) as c\n"
-            + "from emp",
-        "${plan}");
+    final String sql = "select (\n"
+        + " with dept2 as (select * from dept where deptno > 10)"
+        + " select count(*) from dept2) as c\n"
+        + "from emp";
+    sql(sql)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testWithInsideScalarSubqueryRex() {
+    final String sql = "select (\n"
+        + " with dept2 as (select * from dept where deptno > 10)"
+        + " select count(*) from dept2) as c\n"
+        + "from emp";
+    sql(sql)
+        .expand(false)
+        .convertsTo("${plan}");
   }
 
   @Test public void testTableExtend() {
@@ -719,7 +771,8 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
 
   @Test public void testCollectionTableWithCursorParam() {
     tester.withDecorrelation(false).assertConvertsTo(
-        "select * from table(dedup(" + "cursor(select ename from emp),"
+        "select * from table(dedup("
+            + "cursor(select ename from emp),"
             + " cursor(select name from dept), 'NAME'))",
         "${plan}");
   }
@@ -757,18 +810,28 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testMultisetOfColumns() {
-    check(
-        "select 'abc',multiset[deptno,sal] from emp",
-        "${plan}");
+    sql("select 'abc',multiset[deptno,sal] from emp")
+        .expand(true)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testMultisetOfColumnsRex() {
+    sql("select 'abc',multiset[deptno,sal] from emp")
+        .convertsTo("${plan}");
   }
 
   @Test public void testCorrelationJoin() {
-    check(
-        "select *,"
-            + "         multiset(select * from emp where deptno=dept.deptno) "
-            + "               as empset"
-            + "      from dept",
-        "${plan}");
+    final String sql = "select *,\n"
+        + "  multiset(select * from emp where deptno=dept.deptno) as empset\n"
+        + "from dept";
+    sql(sql).convertsTo("${plan}");
+  }
+
+  @Test public void testCorrelationJoinRex() {
+    final String sql = "select *,\n"
+        + "  multiset(select * from emp where deptno=dept.deptno) as empset\n"
+        + "from dept";
+    sql(sql).expand(false).convertsTo("${plan}");
   }
 
   @Test public void testExists() {
@@ -783,24 +846,49 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
         "${plan}");
   }
 
+  @Test public void testNotExistsCorrelated() {
+    final String sql = "select * from emp where not exists (\n"
+        + "  select 1 from dept where emp.deptno=dept.deptno)";
+    tester.withDecorrelation(false).assertConvertsTo(sql, "${plan}");
+  }
+
   @Test public void testExistsCorrelatedDecorrelate() {
-    tester.withDecorrelation(true).assertConvertsTo(
-        "select*from emp where exists (select 1 from dept where emp.deptno=dept.deptno)",
-        "${plan}");
+    final String sql = "select*from emp where exists (\n"
+        + "  select 1 from dept where emp.deptno=dept.deptno)";
+    tester.withDecorrelation(true).assertConvertsTo(sql, "${plan}");
+  }
+
+  @Test public void testExistsCorrelatedDecorrelateRex() {
+    final String sql = "select*from emp where exists (\n"
+        + "  select 1 from dept where emp.deptno=dept.deptno)";
+    tester.withDecorrelation(true).withExpand(false)
+        .assertConvertsTo(sql, "${plan}");
   }
 
   @Test public void testExistsCorrelatedLimit() {
-    tester.withDecorrelation(false).assertConvertsTo(
-        "select*from emp where exists (\n"
-            + "  select 1 from dept where emp.deptno=dept.deptno limit 1)",
-        "${plan}");
+    final String sql = "select*from emp where exists (\n"
+        + "  select 1 from dept where emp.deptno=dept.deptno limit 1)";
+    sql(sql)
+        .decorrelate(false)
+        .convertsTo("${plan}");
   }
 
   @Test public void testExistsCorrelatedLimitDecorrelate() {
-    tester.withDecorrelation(true).assertConvertsTo(
-        "select*from emp where exists (\n"
-            + "  select 1 from dept where emp.deptno=dept.deptno limit 1)",
-        "${plan}");
+    final String sql = "select*from emp where exists (\n"
+        + "  select 1 from dept where emp.deptno=dept.deptno limit 1)";
+    sql(sql)
+        .decorrelate(true)
+        .expand(true)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testExistsCorrelatedLimitDecorrelateRex() {
+    final String sql = "select*from emp where exists (\n"
+        + "  select 1 from dept where emp.deptno=dept.deptno limit 1)";
+    sql(sql)
+        .decorrelate(true)
+        .expand(false)
+        .convertsTo("${plan}");
   }
 
   @Test public void testInValueListShort() {
@@ -816,60 +904,166 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testInUncorrelatedSubquery() {
-    check(
-        "select empno from emp where deptno in"
-            + " (select deptno from dept)",
-        "${plan}");
+    final String sql = "select empno from emp where deptno in"
+        + " (select deptno from dept)";
+    sql(sql)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testInUncorrelatedSubqueryRex() {
+    final String sql = "select empno from emp where deptno in"
+        + " (select deptno from dept)";
+    sql(sql)
+        .expand(false)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testCompositeInUncorrelatedSubqueryRex() {
+    final String sql = "select empno from emp where (empno, deptno) in"
+        + " (select deptno - 10, deptno from dept)";
+    sql(sql)
+        .expand(false)
+        .convertsTo("${plan}");
   }
 
   @Test public void testNotInUncorrelatedSubquery() {
-    check(
-        "select empno from emp where deptno not in"
-            + " (select deptno from dept)",
-        "${plan}");
+    final String sql = "select empno from emp where deptno not in"
+        + " (select deptno from dept)";
+    sql(sql)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testNotInUncorrelatedSubqueryRex() {
+    final String sql = "select empno from emp where deptno not in"
+        + " (select deptno from dept)";
+    sql(sql)
+        .expand(false)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testWhereInCorrelated() {
+    final String sql = "select empno from emp as e\n"
+        + "join dept as d using (deptno)\n"
+        + "where e.sal in (\n"
+        + "  select e2.sal from emp as e2 where e2.deptno > e.deptno)";
+    sql(sql)
+        .expand(false)
+        .convertsTo("${plan}");
   }
 
   @Test public void testInUncorrelatedSubqueryInSelect() {
     // In the SELECT clause, the value of IN remains in 3-valued logic
     // -- it's not forced into 2-valued by the "... IS TRUE" wrapper as in the
     // WHERE clause -- so the translation is more complicated.
-    check(
-        "select name, deptno in (\n"
-            + "  select case when true then deptno else null end from emp)\n"
-            + "from dept",
-        "${plan}");
+    final String sql = "select name, deptno in (\n"
+        + "  select case when true then deptno else null end from emp)\n"
+        + "from dept";
+    sql(sql)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testInUncorrelatedSubqueryInSelectRex() {
+    // In the SELECT clause, the value of IN remains in 3-valued logic
+    // -- it's not forced into 2-valued by the "... IS TRUE" wrapper as in the
+    // WHERE clause -- so the translation is more complicated.
+    final String sql = "select name, deptno in (\n"
+        + "  select case when true then deptno else null end from emp)\n"
+        + "from dept";
+    sql(sql)
+        .expand(false)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testInUncorrelatedSubqueryInHavingRex() {
+    final String sql = "select sum(sal) as s\n"
+        + "from emp\n"
+        + "group by deptno\n"
+        + "having count(*) > 2\n"
+        + "and deptno in (\n"
+        + "  select case when true then deptno else null end from emp)";
+    sql(sql).expand(false).convertsTo("${plan}");
+  }
+
+  @Test public void testUncorrelatedScalarSubqueryInOrderRex() {
+    final String sql = "select ename\n"
+        + "from emp\n"
+        + "order by (select case when true then deptno else null end from emp) desc,\n"
+        + "  ename";
+    sql(sql).expand(false).convertsTo("${plan}");
+  }
+
+  @Test public void testUncorrelatedScalarSubqueryInGroupOrderRex() {
+    final String sql = "select sum(sal) as s\n"
+        + "from emp\n"
+        + "group by deptno\n"
+        + "order by (select case when true then deptno else null end from emp) desc,\n"
+        + "  count(*)";
+    sql(sql).expand(false).convertsTo("${plan}");
+  }
+
+  @Test public void testUncorrelatedScalarSubqueryInAggregateRex() {
+    final String sql = "select sum((select min(deptno) from emp)) as s\n"
+        + "from emp\n"
+        + "group by deptno\n";
+    sql(sql).expand(false).convertsTo("${plan}");
   }
 
   /** Plan should be as {@link #testInUncorrelatedSubqueryInSelect}, but with
    * an extra NOT. Both queries require 3-valued logic. */
   @Test public void testNotInUncorrelatedSubqueryInSelect() {
-    check(
-        "select empno, deptno not in (\n"
-            + "  select case when true then deptno else null end from dept)\n"
-            + "from emp",
-        "${plan}");
+    final String sql = "select empno, deptno not in (\n"
+        + "  select case when true then deptno else null end from dept)\n"
+        + "from emp";
+    sql(sql)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testNotInUncorrelatedSubqueryInSelectRex() {
+    final String sql = "select empno, deptno not in (\n"
+        + "  select case when true then deptno else null end from dept)\n"
+        + "from emp";
+    sql(sql)
+        .expand(false)
+        .convertsTo("${plan}");
   }
 
   /** Since 'deptno NOT IN (SELECT deptno FROM dept)' can not be null, we
    * generate a simpler plan. */
   @Test public void testNotInUncorrelatedSubqueryInSelectNotNull() {
-    check(
-        "select empno, deptno not in (\n"
-            + "  select deptno from dept)\n"
-            + "from emp",
-        "${plan}");
+    final String sql = "select empno, deptno not in (\n"
+        + "  select deptno from dept)\n"
+        + "from emp";
+    sql(sql)
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testNotInUncorrelatedSubqueryInSelectNotNullRex() {
+    final String sql = "select empno, deptno not in (\n"
+        + "  select deptno from dept)\n"
+        + "from emp";
+    sql(sql)
+        .expand(false)
+        .convertsTo("${plan}");
   }
 
   @Test public void testUnnestSelect() {
-    check(
-        "select*from unnest(select multiset[deptno] from dept)",
-        "${plan}");
+    final String sql = "select*from unnest(select multiset[deptno] from dept)";
+    sql(sql).expand(true).convertsTo("${plan}");
+  }
+
+  @Test public void testUnnestSelectRex() {
+    final String sql = "select*from unnest(select multiset[deptno] from dept)";
+    sql(sql).expand(false).convertsTo("${plan}");
   }
 
   @Test public void testJoinUnnest() {
-    check(
-        "select*from dept as d, unnest(multiset[d.deptno * 2])",
-        "${plan}");
+    final String sql = "select*from dept as d, unnest(multiset[d.deptno * 2])";
+    sql(sql).convertsTo("${plan}");
+  }
+
+  @Test public void testJoinUnnestRex() {
+    final String sql = "select*from dept as d, unnest(multiset[d.deptno * 2])";
+    sql(sql).expand(false).convertsTo("${plan}");
   }
 
   @Test public void testLateral() {
@@ -879,9 +1073,16 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testLateralDecorrelate() {
-    tester.withDecorrelation(true).assertConvertsTo(
-        "select * from emp, LATERAL (select * from dept where emp.deptno=dept.deptno)",
-        "${plan}");
+    final String sql = "select * from emp,\n"
+        + " LATERAL (select * from dept where emp.deptno=dept.deptno)";
+    tester.withDecorrelation(true).withExpand(true)
+        .assertConvertsTo(sql, "${plan}");
+  }
+
+  @Test public void testLateralDecorrelateRex() {
+    final String sql = "select * from emp,\n"
+        + " LATERAL (select * from dept where emp.deptno=dept.deptno)";
+    tester.withDecorrelation(true).assertConvertsTo(sql, "${plan}");
   }
 
   @Test public void testNestedCorrelations() {
@@ -894,12 +1095,22 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   }
 
   @Test public void testNestedCorrelationsDecorrelated() {
-    tester.withDecorrelation(true).assertConvertsTo(
-        "select * from (select 2+deptno d2, 3+deptno d3 from emp) e\n"
-            + " where exists (select 1 from (select deptno+1 d1 from dept) d\n"
-            + " where d1=e.d2 and exists (select 2 from (select deptno+4 d4, deptno+5 d5, deptno+6 d6 from dept)\n"
-            + " where d4=d.d1 and d5=d.d1 and d6=e.d3))",
-        "${plan}");
+    final String sql = "select *\n"
+        + "from (select 2+deptno d2, 3+deptno d3 from emp) e\n"
+        + " where exists (select 1 from (select deptno+1 d1 from dept) d\n"
+        + " where d1=e.d2 and exists (select 2 from (select deptno+4 d4, deptno+5 d5, deptno+6 d6 from dept)\n"
+        + " where d4=d.d1 and d5=d.d1 and d6=e.d3))";
+    tester.withDecorrelation(true).withExpand(true)
+        .assertConvertsTo(sql, "${plan}");
+  }
+
+  @Test public void testNestedCorrelationsDecorrelatedRex() {
+    final String sql = "select *\n"
+        + "from (select 2+deptno d2, 3+deptno d3 from emp) e\n"
+        + " where exists (select 1 from (select deptno+1 d1 from dept) d\n"
+        + " where d1=e.d2 and exists (select 2 from (select deptno+4 d4, deptno+5 d5, deptno+6 d6 from dept)\n"
+        + " where d4=d.d1 and d5=d.d1 and d6=e.d3))";
+    tester.withDecorrelation(true).assertConvertsTo(sql, "${plan}");
   }
 
   @Test public void testElement() {
@@ -1432,11 +1643,21 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
    * When de-correlating, push join condition into subquery</a>.
    */
   @Test public void testCorrelationScalarAggAndFilter() {
-    tester.withDecorrelation(true).assertConvertsTo(
-       "SELECT e1.empno FROM emp e1, dept d1 where e1.deptno = d1.deptno\n"
-       + "and e1.deptno < 10 and d1.deptno < 15\n"
-       + "and e1.sal > (select avg(sal) from emp e2 where e1.empno = e2.empno)",
-       "${plan}");
+    final String sql = "SELECT e1.empno\n"
+        + "FROM emp e1, dept d1 where e1.deptno = d1.deptno\n"
+        + "and e1.deptno < 10 and d1.deptno < 15\n"
+        + "and e1.sal > (select avg(sal) from emp e2 where e1.empno = e2.empno)";
+    tester.withDecorrelation(true).withExpand(true)
+        .assertConvertsTo(sql, "${plan}");
+  }
+
+  @Test public void testCorrelationScalarAggAndFilterRex() {
+    final String sql = "SELECT e1.empno\n"
+        + "FROM emp e1, dept d1 where e1.deptno = d1.deptno\n"
+        + "and e1.deptno < 10 and d1.deptno < 15\n"
+        + "and e1.sal > (select avg(sal) from emp e2 where e1.empno = e2.empno)";
+    tester.withDecorrelation(true).withExpand(false)
+        .assertConvertsTo(sql, "${plan}");
   }
 
   /**
@@ -1445,11 +1666,21 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
    * When de-correlating, push join condition into subquery</a>.
    */
   @Test public void testCorrelationExistsAndFilter() {
-    tester.withDecorrelation(true).assertConvertsTo(
-       "SELECT e1.empno FROM emp e1, dept d1 where e1.deptno = d1.deptno\n"
-       + "and e1.deptno < 10 and d1.deptno < 15\n"
-       + "and exists (select * from emp e2 where e1.empno = e2.empno)",
-       "${plan}");
+    final String sql = "SELECT e1.empno\n"
+        + "FROM emp e1, dept d1 where e1.deptno = d1.deptno\n"
+        + "and e1.deptno < 10 and d1.deptno < 15\n"
+        + "and exists (select * from emp e2 where e1.empno = e2.empno)";
+    tester.withDecorrelation(true).withExpand(true)
+        .assertConvertsTo(sql, "${plan}");
+  }
+
+  @Test public void testCorrelationExistsAndFilterRex() {
+    final String sql = "SELECT e1.empno\n"
+        + "FROM emp e1, dept d1 where e1.deptno = d1.deptno\n"
+        + "and e1.deptno < 10 and d1.deptno < 15\n"
+        + "and exists (select * from emp e2 where e1.empno = e2.empno)";
+    tester.withDecorrelation(true)
+        .assertConvertsTo(sql, "${plan}");
   }
 
   /**
@@ -1484,9 +1715,17 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   /** Allows fluent testing. */
   public class Sql {
     private final String sql;
+    private final boolean expand;
+    private final boolean decorrelate;
 
     Sql(String sql) {
+      this(sql, true, true);
+    }
+
+    Sql(String sql, boolean expand, boolean decorrelate) {
       this.sql = sql;
+      this.expand = expand;
+      this.decorrelate = decorrelate;
     }
 
     public void ok() {
@@ -1494,7 +1733,17 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     }
 
     public void convertsTo(String plan) {
-      tester.assertConvertsTo(sql, plan);
+      tester.withExpand(expand)
+          .withDecorrelation(decorrelate)
+          .assertConvertsTo(sql, plan, false);
+    }
+
+    public Sql expand(boolean expand) {
+      return new Sql(sql, expand, decorrelate);
+    }
+
+    public Sql decorrelate(boolean decorrelate) {
+      return new Sql(sql, expand, decorrelate);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index e4d1394..9cb4240 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -92,7 +92,7 @@ public abstract class SqlToRelTestBase {
   }
 
   protected Tester createTester() {
-    return new TesterImpl(getDiffRepos(), true, false, null);
+    return new TesterImpl(getDiffRepos(), false, false, true, null);
   }
 
   /**
@@ -209,6 +209,13 @@ public abstract class SqlToRelTestBase {
     /** Returns a tester that optionally decorrelates queries. */
     Tester withDecorrelation(boolean enable);
 
+    /** Returns a tester that optionally expands sub-queries.
+     * If {@code expand} is false, the plan contains a
+     * {@link org.apache.calcite.rex.RexSubQuery} for each sub-query.
+     *
+     * @see Prepare#THREAD_EXPAND */
+    Tester withExpand(boolean expand);
+
     Tester withCatalogReaderFactory(
         Function<RelDataTypeFactory, Prepare.CatalogReader> factory);
 
@@ -453,6 +460,7 @@ public abstract class SqlToRelTestBase {
     private final DiffRepository diffRepos;
     private final boolean enableDecorrelate;
     private final boolean enableTrim;
+    private final boolean enableExpand;
     private final Function<RelDataTypeFactory, Prepare.CatalogReader>
     catalogReaderFactory;
     private RelDataTypeFactory typeFactory;
@@ -463,15 +471,17 @@ public abstract class SqlToRelTestBase {
      * @param diffRepos Diff repository
      * @param enableDecorrelate Whether to decorrelate
      * @param enableTrim Whether to trim unused fields
+     * @param enableExpand Whether to expand sub-queries
      * @param catalogReaderFactory Function to create catalog reader, or null
      */
     protected TesterImpl(DiffRepository diffRepos, boolean enableDecorrelate,
-        boolean enableTrim,
+        boolean enableTrim, boolean enableExpand,
         Function<RelDataTypeFactory, Prepare.CatalogReader>
             catalogReaderFactory) {
       this.diffRepos = diffRepos;
       this.enableDecorrelate = enableDecorrelate;
       this.enableTrim = enableTrim;
+      this.enableExpand = enableExpand;
       this.catalogReaderFactory = catalogReaderFactory;
     }
 
@@ -495,6 +505,7 @@ public abstract class SqlToRelTestBase {
               catalogReader,
               typeFactory);
       converter.setTrimUnusedFields(true);
+      converter.setExpand(enableExpand);
       final SqlNode validatedQuery = validator.validate(sqlQuery);
       RelRoot root =
           converter.convertQuery(validatedQuery, false, true);
@@ -645,19 +656,30 @@ public abstract class SqlToRelTestBase {
     }
 
     public TesterImpl withDecorrelation(boolean enable) {
-      return this.enableDecorrelate == enable ? this
-          : new TesterImpl(diffRepos, enable, enableTrim, catalogReaderFactory);
+      return this.enableDecorrelate == enable
+          ? this
+          : new TesterImpl(diffRepos, enable, enableTrim, enableExpand,
+              catalogReaderFactory);
     }
 
     public Tester withTrim(boolean enable) {
-      return this.enableTrim == enable ? this
-          : new TesterImpl(diffRepos, enableDecorrelate, enable,
+      return this.enableTrim == enable
+          ? this
+          : new TesterImpl(diffRepos, enableDecorrelate, enable, enableExpand,
+              catalogReaderFactory);
+    }
+
+    public Tester withExpand(boolean expand) {
+      return this.enableExpand == expand
+          ? this
+          : new TesterImpl(diffRepos, enableDecorrelate, enableTrim, expand,
               catalogReaderFactory);
     }
 
     public Tester withCatalogReaderFactory(
         Function<RelDataTypeFactory, Prepare.CatalogReader> factory) {
-      return new TesterImpl(diffRepos, enableDecorrelate, false, factory);
+      return new TesterImpl(diffRepos, enableDecorrelate, false, enableExpand,
+          factory);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 2f4f60e..2584dd2 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -5362,6 +5362,43 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "require alias");
   }
 
+  @Test public void testJoinOnIn() {
+    final String sql = "select * from emp join dept\n"
+        + "on dept.deptno in (select deptno from emp)";
+    sql(sql).ok();
+  }
+
+  @Test public void testJoinOnInCorrelated() {
+    final String sql = "select * from emp as e join dept\n"
+        + "on dept.deptno in (select deptno from emp where deptno < e.deptno)";
+    sql(sql).ok();
+  }
+
+  @Test public void testJoinOnInCorrelatedFails() {
+    final String sql = "select * from emp as e join dept as d\n"
+        + "on d.deptno in (select deptno from emp where deptno < d.^empno^)";
+    sql(sql).fails("Column 'EMPNO' not found in table 'D'");
+  }
+
+  @Test public void testJoinOnExistsCorrelated() {
+    final String sql = "select * from emp as e join dept\n"
+        + "on exists (select 1, 2 from emp where deptno < e.deptno)";
+    sql(sql).ok();
+  }
+
+  @Test public void testJoinOnScalarCorrelated() {
+    final String sql = "select * from emp as e join dept d\n"
+        + "on d.deptno = (select 1 from emp where deptno < e.deptno)";
+    sql(sql).ok();
+  }
+
+  @Test public void testJoinOnScalarFails() {
+    final String sql = "select * from emp as e join dept d\n"
+        + "on d.deptno = (^select 1, 2 from emp where deptno < e.deptno^)";
+    sql(sql).fails(
+        "(?s)Cannot apply '\\$SCALAR_QUERY' to arguments of type '\\$SCALAR_QUERY\\(<RECORDTYPE\\(INTEGER EXPR\\$0, INTEGER EXPR\\$1\\)>\\)'\\. Supported form\\(s\\).*");
+  }
+
   @Test public void testJoinUsingThreeWay() {
     check("select *\n"
         + "from emp as e\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
index bfb1e75..101a6ae 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
@@ -31,38 +31,37 @@ public class EnumerableCorrelateTest {
     tester(true, new JdbcTest.HrSchema())
         .query(
             "select empid, name from emps e where exists (select 1 from depts d where d.deptno=e.deptno)")
-        .explainContains(
-            "EnumerableCalc(expr#0..4=[{inputs}], empid=[$t0], name=[$t2])\n"
-            + "  EnumerableSemiJoin(condition=[=($1, $5)], joinType=[inner])\n"
-            + "    EnumerableTableScan(table=[[s, emps]])\n"
-            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], deptno0=[$t0], $f0=[$t5])\n"
-            + "      EnumerableJoin(condition=[=($0, $1)], joinType=[inner])\n"
-            + "        EnumerableAggregate(group=[{1}])\n"
-            + "          EnumerableTableScan(table=[[s, emps]])\n"
+        .explainContains(""
+            + "EnumerableCalc(expr#0..2=[{inputs}], empid=[$t0], name=[$t2])\n"
+            + "  EnumerableSemiJoin(condition=[=($1, $3)], joinType=[inner])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], proj#0..2=[{exprs}])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "    EnumerableJoin(condition=[=($0, $1)], joinType=[inner])\n"
+            + "      EnumerableAggregate(group=[{1}])\n"
+            + "        EnumerableTableScan(table=[[s, emps]])\n"
+            + "      EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])\n"
             + "        EnumerableTableScan(table=[[s, depts]])")
         .returnsUnordered(
             "empid=100; name=Bill",
             "empid=110; name=Theodore",
             "empid=150; name=Sebastian");
-
   }
 
   @Test public void simpleCorrelate() {
     tester(false, new JdbcTest.HrSchema())
         .query(
             "select empid, name from emps e where exists (select 1 from depts d where d.deptno=e.deptno)")
-        .explainContains(
-            "EnumerableCalc(expr#0..5=[{inputs}], expr#6=[IS NOT NULL($t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
-            + "  EnumerableCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{1}])\n"
+        .explainContains(""
+            + "EnumerableCalc(expr#0..5=[{inputs}], empid=[$t0], name=[$t2])\n"
+            + "  EnumerableCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{1}])\n"
             + "    EnumerableTableScan(table=[[s, emps]])\n"
-            + "    EnumerableAggregate(group=[{}], agg#0=[MIN($0)])\n"
-            + "      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], expr#5=[$cor0], expr#6=[$t5.deptno], expr#7=[=($t0, $t6)], $f0=[$t4], $condition=[$t7])\n"
+            + "    EnumerableAggregate(group=[{0}])\n"
+            + "      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], expr#5=[$cor0], expr#6=[$t5.deptno], expr#7=[=($t0, $t6)], i=[$t4], $condition=[$t7])\n"
             + "        EnumerableTableScan(table=[[s, depts]])")
         .returnsUnordered(
             "empid=100; name=Bill",
             "empid=110; name=Theodore",
             "empid=150; name=Sebastian");
-
   }
 
   private CalciteAssert.AssertThat tester(boolean forceDecorrelate,

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 c9c9342..e45b82c 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -217,6 +217,19 @@ LogicalProject(SAL=[$0])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(SAL=[$0])
+  SemiJoin(condition=[=($1, $2)], joinType=[inner])
+    LogicalFilter(condition=[=($1, 200)])
+      LogicalProject(SAL=[$5], DEPTNO=[$7])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(DEPTNO=[$1])
+      LogicalFilter(condition=[=($0, 100)])
+        LogicalProject(SAL=[$5], DEPTNO=[$7])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testFullOuterJoinSimplificationToLeftOuter">
         <Resource name="sql">
@@ -405,6 +418,15 @@ LogicalProject(DDEPTNO=[$0], DNAME=[$1], C=[$2])
         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![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()])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testPushFilterPastAggWithGroupingSets2">
         <Resource name="sql">
@@ -623,6 +645,13 @@ LogicalProject(DEPTNO=[$0])
     LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(DEPTNO=[8])
+  LogicalFilter(condition=[AND(=($0, 7), =($0, 8))])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testReduceConstantsEliminatesFilter">
         <Resource name="sql">
@@ -3140,6 +3169,14 @@ LogicalProject(ENAME=[$0], R=[$1])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(ENAME=[$0], R=[$1])
+  LogicalFilter(condition=[<($1, 2)])
+    LogicalProject(ENAME=[$1], R=[RANK() OVER (PARTITION BY $7 ORDER BY $5 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testPushFilterWithRankExpr">
         <Resource name="sql">
@@ -3157,6 +3194,14 @@ LogicalProject(ENAME=[$0], R=[$1])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(ENAME=[$0], R=[$1])
+  LogicalFilter(condition=[<($1, 2)])
+    LogicalProject(ENAME=[$1], R=[+(RANK() OVER (PARTITION BY $7 ORDER BY $5 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 1)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testDistinctCount1">
         <Resource name="sql">
@@ -3483,6 +3528,13 @@ LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
     LogicalValues(tuples=[[{ 1, 2 }]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
+  LogicalFilter(condition=[>(+(1, 2), +(3, null))])
+    LogicalValues(tuples=[[{ 1, 2 }]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testExpressionInWindowFunction">
         <Resource name="sql">
@@ -3580,6 +3632,16 @@ LogicalAggregate(group=[{0, 9}])
     LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{0, 9}])
+  LogicalJoin(condition=[<($0, $9)], joinType=[inner])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+      LogicalFilter(condition=[=($0, 10)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testPushAggregateSumThroughJoin">
         <Resource name="sql">
@@ -3927,6 +3989,15 @@ LogicalProject(DEPTNO=[$0])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0])
+  LogicalFilter(condition=[>($1, 1)])
+    LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+      LogicalProject(DEPTNO=[$7])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testPushAggregateFunctionsThroughJoin">
         <Resource name="sql">
@@ -4042,6 +4113,34 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testExpandProjectScalar">
+        <Resource name="sql">
+            <![CDATA[select empno,
+  (select deptno from sales.emp where empno < 20) as d
+from sales.emp]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[$SCALAR_QUERY({
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[$9])
+  LogicalJoin(condition=[true], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])
+      LogicalProject(DEPTNO=[$7])
+        LogicalFilter(condition=[<($0, 20)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testSortJoinTranspose2">
         <Resource name="sql">
             <![CDATA[select * from sales.emp e right join (
@@ -4086,6 +4185,16 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalSort(sort0=[$5], sort1=[$10], dir0=[ASC], dir1=[ASC], fetch=[10])
+    LogicalJoin(condition=[=($7, $9)], joinType=[left])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalProject(DEPTNO=[$0], NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
     </TestCase>
     <TestCase name="testSortJoinTranspose4">
         <Resource name="sql">
@@ -4148,6 +4257,375 @@ LogicalSort(sort0=[$0], dir0=[ASC], fetch=[10])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testExpandProjectIn">
+        <Resource name="sql">
+            <![CDATA[select empno,
+  deptno in (select deptno from sales.emp where empno < 20) as d
+from sales.emp]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[IN($7, {
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[CASE(IS NOT NULL($10), true, false)])
+  LogicalJoin(condition=[=($7, $9)], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}])
+      LogicalProject(DEPTNO=[$0], i=[true])
+        LogicalProject(DEPTNO=[$7])
+          LogicalFilter(condition=[<($0, 20)])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandFilterInComposite">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp
+where (empno, deptno) in (
+  select empno, deptno from sales.emp where empno < 20)
+or emp.sal < 100]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[OR(IN($0, $7, {
+LogicalProject(EMPNO=[$0], DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), <($5, 100))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalFilter(condition=[OR(CASE(IS NOT NULL($11), true, false), <($5, 100))])
+      LogicalJoin(condition=[AND(=($0, $9), =($7, $10))], joinType=[left])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0, 1, 2}])
+          LogicalProject(EMPNO=[$0], DEPTNO=[$1], i=[true])
+            LogicalProject(EMPNO=[$0], DEPTNO=[$7])
+              LogicalFilter(condition=[<($0, 20)])
+                LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandJoinIn">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp left join sales.dept
+on emp.deptno in (select deptno from sales.emp where empno < 20)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalJoin(condition=[IN($7, {
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+    LogicalJoin(condition=[true], joinType=[left])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalJoin(condition=[=($7, $11)], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalAggregate(group=[{0}])
+          LogicalProject(DEPTNO=[$7])
+            LogicalFilter(condition=[<($0, 20)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandFilterIn">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp
+where deptno in (select deptno from sales.emp where empno < 20)
+or emp.sal < 100]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[OR(IN($7, {
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), <($5, 100))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalFilter(condition=[OR(CASE(IS NOT NULL($10), true, false), <($5, 100))])
+      LogicalJoin(condition=[=($7, $9)], joinType=[left])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0, 1}])
+          LogicalProject(DEPTNO=[$0], i=[true])
+            LogicalProject(DEPTNO=[$7])
+              LogicalFilter(condition=[<($0, 20)])
+                LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandJoinExists">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp left join sales.dept
+on exists (select deptno from sales.emp where empno < 20)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalJoin(condition=[EXISTS({
+LogicalFilter(condition=[<($0, 20)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+    LogicalJoin(condition=[true], joinType=[left])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalAggregate(group=[{0}])
+          LogicalProject(i=[true])
+            LogicalFilter(condition=[<($0, 20)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandProjectExists">
+        <Resource name="sql">
+            <![CDATA[select empno,
+  exists (select deptno from sales.emp where empno < 20) as d
+from sales.emp]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[EXISTS({
+LogicalFilter(condition=[<($0, 20)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[CASE(IS NOT NULL($9), true, false)])
+  LogicalJoin(condition=[true], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}])
+      LogicalProject(i=[true])
+        LogicalFilter(condition=[<($0, 20)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandJoinScalar">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp left join sales.dept
+on (select deptno from sales.emp where empno < 20)
+ < (select deptno from sales.emp where empno > 100)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalJoin(condition=[<($SCALAR_QUERY({
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), $SCALAR_QUERY({
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[>($0, 100)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}))], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10], $f0=[$11])
+      LogicalJoin(condition=[<($2, $3)], joinType=[left])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalJoin(condition=[true], joinType=[left])
+          LogicalJoin(condition=[true], joinType=[left])
+            LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+            LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])
+              LogicalProject(DEPTNO=[$7])
+                LogicalFilter(condition=[<($0, 20)])
+                  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+          LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])
+            LogicalProject(DEPTNO=[$7])
+              LogicalFilter(condition=[>($0, 100)])
+                LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandFilterExists">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp
+where exists (select deptno from sales.emp where empno < 20)
+or emp.sal < 100]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[OR(EXISTS({
+LogicalFilter(condition=[<($0, 20)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), <($5, 100))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalFilter(condition=[OR(CASE(IS NOT NULL($9), true, false), <($5, 100))])
+      LogicalJoin(condition=[true], joinType=[left])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0}])
+          LogicalProject(i=[true])
+            LogicalFilter(condition=[<($0, 20)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandProjectInComposite">
+        <Resource name="sql">
+            <![CDATA[select empno, (empno, deptno) in (
+    select empno, deptno from sales.emp where empno < 20) as d
+from sales.emp]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[IN($0, $7, {
+LogicalProject(EMPNO=[$0], DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[CASE(IS NOT NULL($11), true, false)])
+  LogicalJoin(condition=[AND(=($0, $9), =($7, $10))], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1, 2}])
+      LogicalProject(EMPNO=[$0], DEPTNO=[$1], i=[true])
+        LogicalProject(EMPNO=[$0], DEPTNO=[$7])
+          LogicalFilter(condition=[<($0, 20)])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandJoinInComposite">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp left join sales.dept
+on (emp.empno, dept.deptno) in (
+  select empno, deptno from sales.emp where empno < 20)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalJoin(condition=[IN($0, $9, {
+LogicalProject(EMPNO=[$0], DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+    LogicalJoin(condition=[true], joinType=[left])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalJoin(condition=[AND(=($0, $11), =($9, $12))], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalAggregate(group=[{0, 1}])
+          LogicalProject(EMPNO=[$0], DEPTNO=[$7])
+            LogicalFilter(condition=[<($0, 20)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandFilterScalar">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp
+where (select deptno from sales.emp where empno < 20)
+ < (select deptno from sales.emp where empno > 100)
+or emp.sal < 100]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[OR(<($SCALAR_QUERY({
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), $SCALAR_QUERY({
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[>($0, 100)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})), <($5, 100))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$9])
+      LogicalFilter(condition=[OR(<($9, $10), <($5, 100))])
+        LogicalJoin(condition=[true], joinType=[left])
+          LogicalJoin(condition=[true], joinType=[left])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+            LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])
+              LogicalProject(DEPTNO=[$7])
+                LogicalFilter(condition=[<($0, 20)])
+                  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+          LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])
+            LogicalProject(DEPTNO=[$7])
+              LogicalFilter(condition=[>($0, 100)])
+                LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testReduceConstantsDup2">
         <Resource name="sql">
             <![CDATA[select *
@@ -4319,4 +4797,232 @@ LogicalProject(JOB=[$1])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testExpandProjectInNullable">
+        <Resource name="sql">
+            <![CDATA[with e2 as (
+  select empno, case when true then deptno else null end as deptno
+  from sales.emp)
+select empno,
+  deptno in (select deptno from e2 where empno < 20) as d
+from e2]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[IN($1, {
+LogicalProject(DEPTNO=[$1])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalProject(EMPNO=[$0], DEPTNO=[CASE(true, CAST($7):INTEGER, null)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+  LogicalProject(EMPNO=[$0], DEPTNO=[CASE(true, CAST($7):INTEGER, null)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], D=[CASE(=($2, 0), false, IS NOT NULL($5), true, IS NULL($1), null, <($3, $2), null, false)])
+  LogicalJoin(condition=[=($1, $4)], joinType=[left])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalProject(EMPNO=[$0], DEPTNO=[CASE(true, CAST($7):INTEGER, null)])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
+        LogicalProject(DEPTNO=[$1])
+          LogicalFilter(condition=[<($0, 20)])
+            LogicalProject(EMPNO=[$0], DEPTNO=[CASE(true, CAST($7):INTEGER, null)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0, 1}])
+      LogicalProject(DEPTNO=[$0], i=[true])
+        LogicalProject(DEPTNO=[$1])
+          LogicalFilter(condition=[<($0, 20)])
+            LogicalProject(EMPNO=[$0], DEPTNO=[CASE(true, CAST($7):INTEGER, null)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandFilterIn3Value">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp
+where empno
+ < case deptno in (select case when true then deptno else null end
+                   from sales.emp where empno < 20)
+   when true then 10
+   when false then 20
+   else 30
+   end]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[<($0, CASE(=(IN($7, {
+LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), true), 10, =(IN($7, {
+LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), false), 20, 30))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalFilter(condition=[<($0, CASE(=(CASE(=($9, 0), false, IS NOT NULL($12), true, <($10, $9), null, false), true), 10, =(CASE(=($9, 0), false, IS NOT NULL($12), true, <($10, $9), null, false), false), 20, 30))])
+      LogicalJoin(condition=[=($7, $11)], joinType=[left])
+        LogicalJoin(condition=[true], joinType=[inner])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+          LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
+            LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+              LogicalFilter(condition=[<($0, 20)])
+                LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0, 1}])
+          LogicalProject(EXPR$0=[$0], i=[true])
+            LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+              LogicalFilter(condition=[<($0, 20)])
+                LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandFilterExists3Value">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp
+where empno
+  < case exists (select deptno from sales.emp where empno < 20)
+    when true then 10 when false then 20 else 30 end]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[<($0, CASE(=(IN($7, {
+LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), true), 10, =(IN($7, {
+LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+  LogicalFilter(condition=[<($0, 20)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), false), 20, 30))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalFilter(condition=[<($0, CASE(=(CASE(=($9, 0), false, IS NOT NULL($12), true, <($10, $9), null, false), true), 10, =(CASE(=($9, 0), false, IS NOT NULL($12), true, <($10, $9), null, false), false), 20, 30))])
+      LogicalJoin(condition=[=($7, $11)], joinType=[left])
+        LogicalJoin(condition=[true], joinType=[inner])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+          LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
+            LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+              LogicalFilter(condition=[<($0, 20)])
+                LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0, 1}])
+          LogicalProject(EXPR$0=[$0], i=[true])
+            LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+              LogicalFilter(condition=[<($0, 20)])
+                LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandFilterExistsSimple">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp
+where exists (select deptno from sales.emp where empno < 20)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[EXISTS({
+LogicalFilter(condition=[<($0, 20)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{0}])
+        LogicalProject(i=[true])
+          LogicalFilter(condition=[<($0, 20)])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExpandFilterExistsSimpleAnd">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp
+where exists (select deptno from sales.emp where empno < 20)
+and emp.sal < 100]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[AND(EXISTS({
+LogicalFilter(condition=[<($0, 20)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}), <($5, 100))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalFilter(condition=[<($5, 100)])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0}])
+          LogicalProject(i=[true])
+            LogicalFilter(condition=[<($0, 20)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testWhereInCorrelated">
+        <Resource name="sql">
+            <![CDATA[select empno from emp as e
+join dept as d using (deptno)
+where e.sal in (
+  select e2.sal from emp as e2 where e2.deptno > e.deptno)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[IN($5, {
+LogicalProject(SAL=[$5])
+  LogicalFilter(condition=[>($7, $cor0.DEPTNO)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})], variablesSet=[[$cor0]])
+    LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+    LogicalFilter(condition=[=($5, $11)])
+      LogicalCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{7}])
+        LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalAggregate(group=[{0}])
+          LogicalProject(SAL=[$5])
+            LogicalFilter(condition=[>($7, $cor0.DEPTNO)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>


[13/50] [abbrv] calcite git commit: Deprecate some Util methods, and upgrade last modules to JDK 1.7

Posted by jh...@apache.org.
Deprecate some Util methods, and upgrade last modules to JDK 1.7


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

Branch: refs/heads/branch-release
Commit: 4ae0298688d402484be7c76610e46c6f610a8bc6
Parents: 77e6f49
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Sep 10 19:27:49 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/rel/RelRoot.java    |  8 +++
 .../main/java/org/apache/calcite/util/Util.java | 51 ++++++++++++--------
 .../calcite/sql/test/SqlOperatorBaseTest.java   |  5 +-
 .../org/apache/calcite/test/LatticeTest.java    |  4 +-
 .../org/apache/calcite/tools/PlannerTest.java   | 34 +++++++------
 mongodb/pom.xml                                 |  8 ---
 .../calcite/adapter/mongodb/MongoTable.java     | 13 ++---
 plus/pom.xml                                    |  8 ---
 spark/pom.xml                                   |  7 ---
 9 files changed, 71 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/core/src/main/java/org/apache/calcite/rel/RelRoot.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelRoot.java b/core/src/main/java/org/apache/calcite/rel/RelRoot.java
index 77f3782..82506d2 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelRoot.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelRoot.java
@@ -109,6 +109,14 @@ public class RelRoot {
         RelCollations.EMPTY);
   }
 
+  @Override public String toString() {
+    return "Root {kind: " + kind
+        + ", rel: " + rel
+        + ", rowType: " + validatedRowType
+        + ", fields: " + fields
+        + ", collation: " + collation + "}";
+  }
+
   /** Creates a copy of this RelRoot, assigning a {@link RelNode}. */
   public RelRoot withRel(RelNode rel) {
     if (rel == this.rel) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 35dc0ff..4dc2fdc 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -52,6 +52,7 @@ import java.io.StringWriter;
 import java.io.Writer;
 import java.lang.reflect.Array;
 import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.net.MalformedURLException;
 import java.net.URL;
@@ -323,7 +324,7 @@ public class Util {
     } else if (set2.isEmpty()) {
       return set1;
     } else {
-      Set<T> set = new HashSet<T>(set1);
+      Set<T> set = new HashSet<>(set1);
       set.removeAll(set2);
       return set;
     }
@@ -432,7 +433,7 @@ public class Util {
       Field[] fields = clazz.getFields();
       int printed = 0;
       for (Field field : fields) {
-        if (isStatic(field)) {
+        if (Modifier.isStatic(field.getModifiers())) {
           continue;
         }
         if (printed++ > 0) {
@@ -469,8 +470,6 @@ public class Util {
     if (s == null) {
       if (nullMeansNull) {
         pw.print("null");
-      } else {
-        //pw.print("");
       }
     } else {
       String s1 = replace(s, "\\", "\\\\");
@@ -583,6 +582,7 @@ public class Util {
   /**
    * Creates a file-protocol URL for the given file.
    */
+  @Deprecated // to be removed before 2.0
   public static URL toURL(File file) throws MalformedURLException {
     String path = file.getAbsolutePath();
 
@@ -607,6 +607,7 @@ public class Util {
    * Gets a timestamp string for use in file names. The generated timestamp
    * string reflects the current time.
    */
+  @Deprecated // to be removed before 2.0
   public static String getFileTimestamp() {
     SimpleDateFormat sdf = new SimpleDateFormat(FILE_TIMESTAMP_FORMAT);
     return sdf.format(new java.util.Date());
@@ -700,19 +701,15 @@ public class Util {
    * @param iter iterator to materialize
    * @return materialized list
    */
+  @Deprecated // to be removed before 2.0
   public static <T> List<T> toList(Iterator<T> iter) {
-    List<T> list = new ArrayList<T>();
+    List<T> list = new ArrayList<>();
     while (iter.hasNext()) {
       list.add(iter.next());
     }
     return list;
   }
 
-  static boolean isStatic(java.lang.reflect.Member member) {
-    int modifiers = member.getModifiers();
-    return java.lang.reflect.Modifier.isStatic(modifiers);
-  }
-
   /**
    * @return true if s==null or if s.length()==0
    */
@@ -800,6 +797,7 @@ public class Util {
    *
    * @return a non-null string containing all messages of the exception
    */
+  @Deprecated // to be removed before 2.0
   public static String getMessages(Throwable t) {
     StringBuilder sb = new StringBuilder();
     for (Throwable curr = t; curr != null; curr = curr.getCause()) {
@@ -820,7 +818,10 @@ public class Util {
    *
    * @param t Throwable
    * @return Stack trace
+   *
+   * @deprecated Use {@link com.google.common.base.Throwables#getStackTraceAsString(Throwable)}
    */
+  @Deprecated // to be removed before 2.0
   public static String getStackTrace(Throwable t) {
     final StringWriter sw = new StringWriter();
     final PrintWriter pw = new PrintWriter(sw);
@@ -1010,6 +1011,7 @@ public class Util {
    * @param reader reader to read from
    * @return reader contents as string
    */
+  @Deprecated // to be removed before 2.0
   public static String readAllAsString(Reader reader) throws IOException {
     StringBuilder sb = new StringBuilder();
     char[] buf = new char[4096];
@@ -1030,6 +1032,7 @@ public class Util {
    *
    * @param jar jar to close
    */
+  @Deprecated // to be removed before 2.0
   public static void squelchJar(JarFile jar) {
     try {
       if (jar != null) {
@@ -1047,6 +1050,7 @@ public class Util {
    *
    * @param stream stream to close
    */
+  @Deprecated // to be removed before 2.0
   public static void squelchStream(InputStream stream) {
     try {
       if (stream != null) {
@@ -1066,6 +1070,7 @@ public class Util {
    *
    * @param stream stream to close
    */
+  @Deprecated // to be removed before 2.0
   public static void squelchStream(OutputStream stream) {
     try {
       if (stream != null) {
@@ -1083,6 +1088,7 @@ public class Util {
    *
    * @param reader reader to close
    */
+  @Deprecated // to be removed before 2.0
   public static void squelchReader(Reader reader) {
     try {
       if (reader != null) {
@@ -1102,6 +1108,7 @@ public class Util {
    *
    * @param writer writer to close
    */
+  @Deprecated // to be removed before 2.0
   public static void squelchWriter(Writer writer) {
     try {
       if (writer != null) {
@@ -1119,6 +1126,7 @@ public class Util {
    *
    * @param stmt stmt to close
    */
+  @Deprecated // to be removed before 2.0
   public static void squelchStmt(Statement stmt) {
     try {
       if (stmt != null) {
@@ -1136,6 +1144,7 @@ public class Util {
    *
    * @param connection connection to close
    */
+  @Deprecated // to be removed before 2.0
   public static void squelchConnection(Connection connection) {
     try {
       if (connection != null) {
@@ -1152,6 +1161,7 @@ public class Util {
    * @param s string to be trimmed
    * @return trimmed string
    */
+  @Deprecated // to be removed before 2.0
   public static String rtrim(String s) {
     int n = s.length() - 1;
     if (n >= 0) {
@@ -1174,6 +1184,7 @@ public class Util {
    * @param len desired length
    * @return padded string
    */
+  @Deprecated // to be removed before 2.0
   public static String rpad(String s, int len) {
     if (s.length() >= len) {
       return s;
@@ -1509,6 +1520,7 @@ public class Util {
    * @throws IOException
    * @throws InterruptedException
    */
+  @Deprecated // to be removed before 2.0
   public static int runApplication(
       String[] cmdarray,
       Logger logger,
@@ -1527,6 +1539,7 @@ public class Util {
    * @param cmdarray command and arguments.
    * @return a ProcessBuilder.
    */
+  @Deprecated // to be removed before 2.0
   public static ProcessBuilder newAppProcess(String[] cmdarray) {
     // Concatenate quoted words from cmdarray.
     // REVIEW mb 2/24/09 Why is this needed?
@@ -1702,11 +1715,11 @@ public class Util {
    * @param includeFilter Class whose instances to include
    */
   public static <E> Iterable<E> filter(
-      final Iterable<? extends Object> iterable,
+      final Iterable<?> iterable,
       final Class<E> includeFilter) {
     return new Iterable<E>() {
       public Iterator<E> iterator() {
-        return new Filterator<E>(iterable.iterator(), includeFilter);
+        return new Filterator<>(iterable.iterator(), includeFilter);
       }
     };
   }
@@ -1718,7 +1731,7 @@ public class Util {
       private int size = -1;
 
       public Iterator<E> iterator() {
-        return new Filterator<E>(collection.iterator(), includeFilter);
+        return new Filterator<>(collection.iterator(), includeFilter);
       }
 
       public int size() {
@@ -1728,9 +1741,7 @@ public class Util {
           // filtering values.  (Some java.util algorithms
           // call next() on the result of iterator() size() times.)
           int s = 0;
-          Iterator<E> iter = iterator();
-          while (iter.hasNext()) {
-            iter.next();
+          for (E e : this) {
             s++;
           }
           size = s;
@@ -1753,7 +1764,7 @@ public class Util {
   public static <E> List<E> filter(
       final List<?> list,
       final Class<E> includeFilter) {
-    List<E> result = new ArrayList<E>();
+    List<E> result = new ArrayList<>();
     for (Object o : list) {
       if (includeFilter.isInstance(o)) {
         result.add(includeFilter.cast(o));
@@ -1784,6 +1795,7 @@ public class Util {
    */
   public static Map<String, String> toMap(
       final Properties properties) {
+    //noinspection unchecked
     return (Map) properties;
   }
 
@@ -1806,7 +1818,7 @@ public class Util {
    * @return Map with given contents
    */
   public static <K, V> Map<K, V> mapOf(K key, V value, Object... keyValues) {
-    final Map<K, V> map = new LinkedHashMap<K, V>(1 + keyValues.length);
+    final Map<K, V> map = new LinkedHashMap<>(1 + keyValues.length);
     map.put(key, value);
     for (int i = 0; i < keyValues.length;) {
       //noinspection unchecked
@@ -2071,7 +2083,7 @@ public class Util {
       }
       return -1;
     }
-    final Map<E, Object> set = new HashMap<E, Object>(size);
+    final Map<E, Object> set = new HashMap<>(size);
     for (E e : list) {
       if (set.put(e, "") != null) {
         return set.size();
@@ -2316,6 +2328,7 @@ public class Util {
     private final Object node;
 
     /** Singleton instance. Can be used if you don't care about node. */
+    @SuppressWarnings("ThrowableInstanceNeverThrown")
     public static final FoundOne NULL = new FoundOne(null);
 
     public FoundOne(Object node) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/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 47161b3..6a35641 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
@@ -50,6 +50,7 @@ import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
+import com.google.common.base.Throwables;
 import com.google.common.collect.Lists;
 
 import org.junit.Before;
@@ -5274,7 +5275,7 @@ public abstract class SqlOperatorBaseTest {
       } catch (SQLException e) {
         thrown = e;
       }
-      final String stack = Util.getStackTrace(thrown);
+      final String stack = Throwables.getStackTraceAsString(thrown);
       for (Pattern pattern : patterns) {
         if (pattern.matcher(stack).matches()) {
           return;
@@ -5316,7 +5317,7 @@ public abstract class SqlOperatorBaseTest {
         thrown = e;
       }
       if (thrown != null) {
-        final String stack = Util.getStackTrace(thrown);
+        final String stack = Throwables.getStackTraceAsString(thrown);
         for (Pattern pattern : patterns) {
           if (pattern.matcher(stack).matches()) {
             return;

http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/core/src/test/java/org/apache/calcite/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
index 69eea60..397ad3a 100644
--- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
@@ -25,6 +25,7 @@ import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -236,7 +237,8 @@ public class LatticeTest {
                       + "    StarTableScan(table=[[adhoc, star]])\n",
                   counter));
     } catch (RuntimeException e) {
-      assertThat(Util.getStackTrace(e), containsString("CannotPlanException"));
+      assertThat(Throwables.getStackTraceAsString(e),
+          containsString("CannotPlanException"));
     }
     assertThat(counter.get(), equalTo(1));
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index b769653..fae8cb9 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -38,6 +38,7 @@ import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.TableScan;
@@ -73,6 +74,7 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.test.CalciteAssert;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Ignore;
@@ -102,7 +104,7 @@ public class PlannerTest {
     assertThat(Util.toLinux(parse.toString()), equalTo(queryFromParseTree));
 
     SqlNode validate = planner.validate(parse);
-    RelNode rel = planner.convert(validate);
+    RelNode rel = planner.rel(validate).project();
     assertThat(toString(rel), equalTo(expectedRelExpr));
   }
 
@@ -184,7 +186,7 @@ public class PlannerTest {
       SqlNode validate = planner.validate(parse);
       fail("expected error, got " + validate);
     } catch (ValidationException e) {
-      assertThat(Util.getStackTrace(e),
+      assertThat(Throwables.getStackTraceAsString(e),
           containsString("Column 'Xname' not found in any table"));
       // ok
     }
@@ -256,7 +258,7 @@ public class PlannerTest {
     Planner planner = getPlanner(null);
     SqlNode parse = planner.parse("select * from \"emps\"");
     try {
-      RelNode rel = planner.convert(parse);
+      RelRoot rel = planner.rel(parse);
       fail("expected error, got " + rel);
     } catch (IllegalArgumentException e) {
       assertThat(e.getMessage(),
@@ -272,7 +274,7 @@ public class PlannerTest {
     Planner planner = getPlanner(null);
     SqlNode parse = planner.parse(sql);
     SqlNode validate = planner.validate(parse);
-    RelNode rel = planner.convert(validate);
+    RelNode rel = planner.rel(validate).project();
     final RelOptPredicateList predicates =
         RelMetadataQuery.getPulledUpPredicates(rel);
     final String buf = predicates.pulledUpPredicates.toString();
@@ -327,7 +329,7 @@ public class PlannerTest {
     Planner planner = getPlanner(null, program);
     SqlNode parse = planner.parse("select * from \"emps\"");
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).project();
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -350,7 +352,7 @@ public class PlannerTest {
         "select * from \"emps\" "
             + "order by \"emps\".\"deptno\"");
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).project();
     RelTraitSet traitSet = convert.getTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -432,7 +434,7 @@ public class PlannerTest {
         Programs.of(ruleSet));
     SqlNode parse = planner.parse(sql);
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).rel;
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     if (traitSet.getTrait(RelCollationTraitDef.INSTANCE) == null) {
@@ -458,7 +460,7 @@ public class PlannerTest {
             + "order by \"emps\".\"deptno\") "
             + "order by \"deptno\"");
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).rel;
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -486,7 +488,7 @@ public class PlannerTest {
 
     SqlNode parse = planner.parse("select * from \"emps\"");
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).project();
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -506,7 +508,7 @@ public class PlannerTest {
     Planner planner = getPlanner(null, Programs.of(ruleSet));
     SqlNode parse = planner.parse("select * from \"emps\"");
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).project();
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -609,7 +611,7 @@ public class PlannerTest {
     SqlNode parse = planner.parse("select T1.\"name\" from \"emps\" as T1 ");
 
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).project();
 
     RelTraitSet traitSet0 = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
@@ -670,7 +672,7 @@ public class PlannerTest {
     SqlNode parse = planner.parse(buf.toString());
 
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).project();
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -749,7 +751,7 @@ public class PlannerTest {
         Programs.heuristicJoinOrder(Programs.RULE_SET, false, 0));
     SqlNode parse = planner.parse(sql);
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).rel;
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -877,7 +879,7 @@ public class PlannerTest {
     SqlNode parse = planner.parse(sql);
 
     SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.convert(validate);
+    RelNode convert = planner.rel(validate).project();
     RelTraitSet traitSet = planner.getEmptyTraitSet()
         .replace(EnumerableConvention.INSTANCE);
     RelNode transform = planner.transform(0, traitSet, convert);
@@ -992,7 +994,7 @@ public class PlannerTest {
     Planner p = Frameworks.getPlanner(config);
     SqlNode n = p.parse(tpchTestQuery);
     n = p.validate(n);
-    RelNode r = p.convert(n);
+    RelNode r = p.rel(n).project();
     String plan = RelOptUtil.toString(r);
     p.close();
     return plan;
@@ -1050,7 +1052,7 @@ public class PlannerTest {
             .build());
     SqlNode n = p.parse(query);
     n = p.validate(n);
-    RelNode r = p.convert(n);
+    RelNode r = p.rel(n).project();
     String plan = RelOptUtil.toString(r);
     plan = Util.toLinux(plan);
     p.close();

http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/mongodb/pom.xml b/mongodb/pom.xml
index 96c8adb..5d23fc2 100644
--- a/mongodb/pom.xml
+++ b/mongodb/pom.xml
@@ -78,14 +78,6 @@ limitations under the License.
            pluginManagement in the root POM, not here. -->
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.5</source>
-          <target>1.5</target>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <executions>
           <execution>

http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java
index af8d80c..78e1898 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoTable.java
@@ -34,6 +34,8 @@ import org.apache.calcite.schema.impl.AbstractTableQueryable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Util;
 
+import com.google.common.collect.Lists;
+
 import com.mongodb.AggregationOptions;
 import com.mongodb.AggregationOutput;
 import com.mongodb.BasicDBList;
@@ -76,7 +78,7 @@ public class MongoTable extends AbstractQueryableTable
 
   public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
       SchemaPlus schema, String tableName) {
-    return new MongoQueryable<T>(queryProvider, schema, this, tableName);
+    return new MongoQueryable<>(queryProvider, schema, this, tableName);
   }
 
   public RelNode toRel(
@@ -131,7 +133,7 @@ public class MongoTable extends AbstractQueryableTable
   public Enumerable<Object> aggregate(final DB mongoDb,
       final List<Map.Entry<String, Class>> fields,
       final List<String> operations) {
-    final List<DBObject> list = new ArrayList<DBObject>();
+    final List<DBObject> list = new ArrayList<>();
     final BasicDBList versionArray = (BasicDBList) mongoDb
         .command("buildInfo").get("versionArray");
     final Integer versionMajor = parseIntString(versionArray
@@ -165,10 +167,9 @@ public class MongoTable extends AbstractQueryableTable
                    .outputMode(AggregationOptions.OutputMode.CURSOR).build();
               // Warning - this can result in a very large ArrayList!
               // but you should know your data and aggregate accordingly
-              ArrayList<DBObject> resultAsArrayList
-                = new ArrayList<DBObject>(Util.toList(mongoDb.
-                      getCollection(collectionName)
-                       .aggregate(list, options)));
+              final List<DBObject> resultAsArrayList =
+                  Lists.newArrayList(mongoDb.getCollection(collectionName)
+                      .aggregate(list, options));
               resultIterator = resultAsArrayList.iterator();
             } else { // Pre MongoDB version 2.6
               AggregationOutput result = aggregateOldWay(mongoDb

http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/plus/pom.xml
----------------------------------------------------------------------
diff --git a/plus/pom.xml b/plus/pom.xml
index 8975280..99721ce 100644
--- a/plus/pom.xml
+++ b/plus/pom.xml
@@ -86,14 +86,6 @@ limitations under the License.
            pluginManagement in the root POM, not here. -->
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.5</source>
-          <target>1.5</target>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <executions>
           <execution>

http://git-wip-us.apache.org/repos/asf/calcite/blob/4ae02986/spark/pom.xml
----------------------------------------------------------------------
diff --git a/spark/pom.xml b/spark/pom.xml
index 66dd23b..25448ca 100644
--- a/spark/pom.xml
+++ b/spark/pom.xml
@@ -91,13 +91,6 @@ limitations under the License.
   <build>
     <plugins>
       <plugin>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.5</source>
-          <target>1.5</target>
-        </configuration>
-      </plugin>
-      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <executions>


[43/50] [abbrv] calcite git commit: [CALCITE-1047] ChunkList.clear throws AssertionError

Posted by jh...@apache.org.
[CALCITE-1047] ChunkList.clear throws AssertionError

Re-work ChunkList with a more robust implementation of ListIterator and more thorough tests.


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

Branch: refs/heads/branch-release
Commit: 3cba7055061108ce0e079d8756bcfb020b8628c3
Parents: 2712d7d
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jan 12 20:14:51 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 13 08:43:29 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/util/ChunkList.java | 243 ++++++++++++-------
 .../org/apache/calcite/util/ChunkListTest.java  | 217 ++++++++++++++---
 2 files changed, 340 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/3cba7055/core/src/main/java/org/apache/calcite/util/ChunkList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ChunkList.java b/core/src/main/java/org/apache/calcite/util/ChunkList.java
index 66644e9..8d7c9bf 100644
--- a/core/src/main/java/org/apache/calcite/util/ChunkList.java
+++ b/core/src/main/java/org/apache/calcite/util/ChunkList.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.util;
 
 import java.util.AbstractSequentialList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.ListIterator;
@@ -33,10 +34,6 @@ import java.util.NoSuchElementException;
  */
 public class ChunkList<E> extends AbstractSequentialList<E> {
   private static final int HEADER_SIZE = 3;
-  private int size;
-  private Object[] first;
-  private Object[] last;
-
   private static final int CHUNK_SIZE = 64;
   private static final Integer[] INTEGERS = new Integer[CHUNK_SIZE + 3];
 
@@ -46,6 +43,10 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
     }
   }
 
+  private int size;
+  private Object[] first;
+  private Object[] last;
+
   /**
    * Creates an empty ChunkList.
    */
@@ -105,6 +106,12 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
     return size;
   }
 
+  @Override public void clear() {
+    // base class method works, but let's optimize
+    size = 0;
+    first = last = null;
+  }
+
   @Override public boolean add(E element) {
     Object[] chunk = last;
     int occupied;
@@ -169,18 +176,20 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
   }
 
   private ChunkListIterator locate(int index) {
-    if (index == 0) {
-      return new ChunkListIterator();
+    if (index < 0 || index > size) {
+      throw new IndexOutOfBoundsException();
+    }
+    if (first == null) {
+      // Create an iterator positioned before the first element.
+      return new ChunkListIterator(null, 0, 0, -1, 0);
     }
     int n = 0;
     for (Object[] chunk = first;;) {
       final int occupied = occupied(chunk);
       final int nextN = n + occupied;
       final Object[] next = next(chunk);
-      if (nextN > index || next == null) {
-        return new ChunkListIterator(
-            chunk, n, index - n - 1 + HEADER_SIZE,
-            occupied + HEADER_SIZE);
+      if (nextN >= index || next == null) {
+        return new ChunkListIterator(chunk, n, index, -1, n + occupied);
       }
       n = nextN;
       chunk = next;
@@ -190,76 +199,87 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
   /** Iterator over a {@link ChunkList}. */
   private class ChunkListIterator implements ListIterator<E> {
     private Object[] chunk;
-    private int startIndex;
-    private int offset;
+    /** Offset in the list of the first element of this chunk. */
+    private int start;
+    /** Offset within current chunk of the next element to return. */
+    private int cursor;
+    /** Offset within the current chunk of the last element returned. -1 if
+     * {@link #next} or {@link #previous()} has not been called. */
+    private int lastRet;
+    /** Offset of the first unoccupied location in the current chunk. */
     private int end;
 
-    ChunkListIterator() {
-      this(null, 0, -1, 0);
-    }
-
-    ChunkListIterator(Object[] chunk, int startIndex, int offset, int end) {
+    ChunkListIterator(Object[] chunk, int start, int cursor, int lastRet,
+        int end) {
       this.chunk = chunk;
-      this.startIndex = startIndex;
-      this.offset = offset;
+      this.start = start;
+      this.cursor = cursor;
+      this.lastRet = lastRet;
       this.end = end;
     }
 
     public boolean hasNext() {
-      return offset + 1 < end
-          || (chunk == null
-          ? first != null
-          : ChunkList.next(chunk) != null);
+      return cursor < size;
     }
 
     public E next() {
-      ++offset;
-      assert offset <= end;
-      if (offset == end) {
+      if (cursor >= size) {
+        throw new NoSuchElementException();
+      }
+      if (cursor == end) {
         if (chunk == null) {
           chunk = first;
         } else {
           chunk = ChunkList.next(chunk);
-          startIndex += end - HEADER_SIZE;
         }
+        start = end;
         if (chunk == null) {
-          throw new NoSuchElementException();
+          end = start;
+        } else {
+          end = start + occupied(chunk);
         }
-        offset = HEADER_SIZE;
-        end = occupied(chunk) + HEADER_SIZE;
       }
-      return (E) element(chunk, offset);
+      @SuppressWarnings("unchecked")
+      final E element = (E) element(chunk,
+          HEADER_SIZE + (lastRet = cursor++) - start);
+      return element;
     }
 
     public boolean hasPrevious() {
-      return offset >= HEADER_SIZE || ChunkList.prev(chunk) != null;
+      return cursor > 0;
     }
 
     public E previous() {
-      --offset;
-      if (offset == HEADER_SIZE - 1) {
+      lastRet = cursor--;
+      if (cursor < start) {
         chunk = chunk == null ? last : ChunkList.prev(chunk);
         if (chunk == null) {
           throw new NoSuchElementException();
         }
-        end = occupied(chunk);
-        startIndex -= end;
-        offset = end - 1;
+        final int o = occupied(chunk);
+        end = start;
+        start -= o;
+        assert cursor == end - 1;
       }
-      return (E) element(chunk, offset);
+      //noinspection unchecked
+      return (E) element(chunk, cursor - start);
     }
 
     public int nextIndex() {
-      return startIndex + (offset - HEADER_SIZE) + 1;
+      return cursor;
     }
 
     public int previousIndex() {
-      return startIndex + (offset - HEADER_SIZE);
+      return cursor - 1;
     }
 
     public void remove() {
+      if (lastRet < 0) {
+        throw new IllegalStateException();
+      }
       --size;
-      if (end == HEADER_SIZE + 1) {
+      --cursor;
+      if (end == start + 1) {
         // Chunk is now empty.
         final Object[] prev = prev(chunk);
         final Object[] next = ChunkList.next(chunk);
@@ -272,80 +292,125 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
           }
           chunk = null;
           end = HEADER_SIZE;
-          offset = end - 1;
         } else {
           if (prev == null) {
-            first = next;
+            chunk = first = next;
             setPrev(next, null);
+            end = occupied(chunk);
           } else {
             setNext(prev, next);
             setPrev(next, prev);
+            chunk = prev;
+            end = start;
+            start -= occupied(chunk);
           }
-          chunk = next;
-          offset = HEADER_SIZE;
-          end = HEADER_SIZE + occupied(next);
         }
+        lastRet = -1;
         return;
       }
-      // Move existing contents down one.
-      System.arraycopy(
-          chunk, offset + 1, chunk, offset, end - offset - 1);
-      --end;
-      setElement(chunk, end, null); // allow gc
-      setOccupied(chunk, end - HEADER_SIZE);
-      if (offset == end) {
-        final Object[] next = ChunkList.next(chunk);
-        if (next != null) {
-          startIndex += end - HEADER_SIZE;
-          chunk = next;
-          offset = HEADER_SIZE - 1;
-          end = HEADER_SIZE + occupied(next);
+      final int r = lastRet;
+      lastRet = -1;
+      if (r < start) {
+        // Element we wish to eliminate is the last element in the previous
+        // block.
+        Object[] c = chunk;
+        if (c == null) {
+          c = last;
         }
+        int o = occupied(c);
+        if (o == 1) {
+          // Block is now empty; remove it
+          final Object[] prev = prev(c);
+          if (prev == null) {
+            if (chunk == null) {
+              first = last = null;
+            } else {
+              first = chunk;
+              setPrev(chunk, null);
+            }
+          } else {
+            setNext(prev, chunk);
+            setPrev(chunk, prev);
+          }
+        } else {
+          --o;
+          setElement(c, HEADER_SIZE + o, null); // allow gc
+          setOccupied(c, o);
+        }
+      } else {
+        // Move existing contents down one.
+        System.arraycopy(chunk, HEADER_SIZE + r - start + 1,
+            chunk, HEADER_SIZE + r - start, end - r - 1);
+        --end;
+        final int o = end - start;
+        setElement(chunk, HEADER_SIZE + o, null); // allow gc
+        setOccupied(chunk, o);
       }
     }
 
     public void set(E e) {
-      setElement(chunk, offset, e);
+      if (lastRet < 0) {
+        throw new IllegalStateException();
+      }
+      Object[] c = chunk;
+      int p = lastRet;
+      int s = start;
+      if (p < start) {
+        // The element is at the end of the previous chunk
+        c = prev(c);
+        s -= occupied(c);
+      }
+      setElement(c, HEADER_SIZE + p - s, e);
     }
 
     public void add(E e) {
-      if (chunk == null || end == CHUNK_SIZE + HEADER_SIZE) {
+      if (chunk == null) {
+        Object[] newChunk = new Object[CHUNK_SIZE + HEADER_SIZE];
+        if (first != null) {
+          setNext(newChunk, first);
+          setPrev(first, newChunk);
+        }
+        first = newChunk;
+        if (last == null) {
+          last = newChunk;
+        }
+        chunk = newChunk;
+        end = start;
+      } else if (end == start + CHUNK_SIZE) {
         // FIXME We create a new chunk, but the next chunk might be
         // less than half full. We should consider using it.
         Object[] newChunk = new Object[CHUNK_SIZE + HEADER_SIZE];
-        if (chunk == null) {
-          if (first != null) {
-            setNext(newChunk, first);
-            setPrev(first, newChunk);
-          }
-          first = newChunk;
-          if (last == null) {
-            last = newChunk;
-          }
+        final Object[] next = ChunkList.next(chunk);
+        setPrev(newChunk, chunk);
+        setNext(chunk, newChunk);
+
+        if (next == null) {
+          last = newChunk;
         } else {
-          final Object[] next = ChunkList.next(chunk);
-          setPrev(newChunk, chunk);
-          setNext(chunk, newChunk);
+          setPrev(next, newChunk);
+          setNext(newChunk, next);
+        }
 
-          if (next == null) {
-            last = newChunk;
-          } else {
-            setPrev(next, newChunk);
-            setNext(newChunk, next);
-          }
-          startIndex += CHUNK_SIZE;
+        setOccupied(chunk, CHUNK_SIZE / 2);
+        setOccupied(newChunk, CHUNK_SIZE / 2);
+        System.arraycopy(chunk, HEADER_SIZE + CHUNK_SIZE / 2,
+            newChunk, HEADER_SIZE, CHUNK_SIZE / 2);
+        Arrays.fill(chunk, HEADER_SIZE + CHUNK_SIZE / 2,
+            HEADER_SIZE + CHUNK_SIZE, null);
+
+        if (cursor - start < CHUNK_SIZE / 2) {
+          end -= CHUNK_SIZE / 2;
+        } else {
+          start += CHUNK_SIZE / 2;
+          chunk = newChunk;
         }
-        chunk = newChunk;
-        end = offset = HEADER_SIZE;
-      } else {
-        // Move existing contents up one.
-        System.arraycopy(
-            chunk, offset, chunk, offset + 1, end - offset);
       }
-      setElement(chunk, offset, e);
-//            ++offset;
+      // Move existing contents up one.
+      System.arraycopy(chunk, HEADER_SIZE + cursor - start,
+          chunk, HEADER_SIZE + cursor - start + 1, end - cursor);
       ++end;
-      setOccupied(chunk, end - HEADER_SIZE);
+      setElement(chunk, HEADER_SIZE + cursor - start, e);
+      setOccupied(chunk, end - start);
       ++size;
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/3cba7055/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/ChunkListTest.java b/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
index 0e2bb2d..efb1b09 100644
--- a/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
+++ b/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
@@ -19,6 +19,8 @@ package org.apache.calcite.util;
 import org.apache.calcite.linq4j.function.Function0;
 import org.apache.calcite.linq4j.function.Function1;
 
+import com.google.common.collect.ImmutableList;
+
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -30,9 +32,11 @@ import java.util.List;
 import java.util.ListIterator;
 import java.util.Random;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -44,8 +48,13 @@ public class ChunkListTest {
    * Unit test for {@link ChunkList}.
    */
   @Test public void testChunkList() {
-    final ChunkList<Integer> list = new ChunkList<Integer>();
+    final ChunkList<Integer> list = new ChunkList<>();
+    final ChunkList<Integer> list0 = new ChunkList<>(list);
+    final ChunkList<Integer> list1 = new ChunkList<>(list);
+    list1.add(123);
     assertEquals(0, list.size());
+    assertEquals(0, list0.size());
+    assertEquals(1, list1.size());
     assertTrue(list.isEmpty());
     assertEquals("[]", list.toString());
 
@@ -56,6 +65,20 @@ public class ChunkListTest {
       // ok
     }
 
+    try {
+      list.get(-1);
+      fail("expected exception");
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+
+    try {
+      list.get(0);
+      fail("expected exception");
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+
     list.add(7);
     assertEquals(1, list.size());
     assertEquals(7, (int) list.get(0));
@@ -76,9 +99,10 @@ public class ChunkListTest {
     assertTrue(list.contains(9));
     assertFalse(list.contains(8));
 
-    list.addAll(Collections.nCopies(40, 1));
-    assertEquals(44, list.size());
+    list.addAll(Collections.nCopies(70, 1));
+    assertEquals(74, list.size());
     assertEquals(1, (int) list.get(40));
+    assertEquals(1, (int) list.get(70));
 
     int n = 0;
     for (Integer integer : list) {
@@ -96,44 +120,142 @@ public class ChunkListTest {
     i = list.indexOf(null);
     assertEquals(-1, i);
 
+    // sort an empty list
+    Collections.sort(list0);
+    assertThat(list0.isEmpty(), is(true));
+
+    // sort a list with 1 element
+    Collections.sort(list1);
+    assertThat(list1.size(), is(1));
+
     Collections.sort(list);
+    assertEquals(74, list.size());
 
     list.remove((Integer) 7);
     Collections.sort(list);
     assertEquals(1, (int) list.get(3));
 
     // remove all instances of a value that exists
-    boolean b = list.removeAll(Arrays.asList(9));
+    boolean b = list.removeAll(Collections.singletonList(9));
     assertTrue(b);
 
     // remove all instances of a non-existent value
-    b = list.removeAll(Arrays.asList(99));
+    b = list.removeAll(Collections.singletonList(99));
     assertFalse(b);
 
     // remove all instances of a value that occurs in the last chunk
     list.add(12345);
-    b = list.removeAll(Arrays.asList(12345));
+    b = list.removeAll(Collections.singletonList(12345));
     assertTrue(b);
 
     // remove all instances of a value that occurs in the last chunk but
     // not as the last value
     list.add(12345);
     list.add(123);
-    b = list.removeAll(Arrays.asList(12345));
+    b = list.removeAll(Collections.singletonList(12345));
     assertTrue(b);
 
-    assertEquals(
-        1000, new ChunkList<Integer>(Collections.nCopies(1000, 77)).size());
+    assertThat(new ChunkList<>(Collections.nCopies(1000, 77)).size(),
+        is(1000));
 
     // add to an empty list via iterator
     //noinspection MismatchedQueryAndUpdateOfCollection
-    final ChunkList<String> list2 = new ChunkList<String>();
+    final ChunkList<String> list2 = new ChunkList<>();
     list2.listIterator(0).add("x");
     assertEquals("[x]", list2.toString());
 
     // add at start
     list2.add(0, "y");
     assertEquals("[y, x]", list2.toString());
+
+    list2.remove(0);
+    assertEquals("[x]", list2.toString());
+
+    // clear a list of length 5, one element at a time, using an iterator
+    list2.clear();
+    list2.addAll(ImmutableList.of("a", "b", "c", "d", "e"));
+    assertThat(list2.size(), is(5));
+    final ListIterator<String> listIterator = list2.listIterator(0);
+    assertThat(listIterator.next(), is("a"));
+    listIterator.remove();
+    assertThat(listIterator.next(), is("b"));
+    listIterator.remove();
+    assertThat(listIterator.next(), is("c"));
+    listIterator.remove();
+    assertThat(listIterator.next(), is("d"));
+    listIterator.remove();
+    assertThat(list2.size(), is(1));
+    assertThat(listIterator.next(), is("e"));
+    listIterator.remove();
+    assertThat(list2.size(), is(0));
+  }
+
+  /** Clears lists of various sizes. */
+  @Test public void testClear() {
+    checkListClear(0);
+    checkListClear(1);
+    checkListClear(2);
+    checkListClear(32);
+    checkListClear(64);
+    checkListClear(65);
+    checkListClear(66);
+    checkListClear(100);
+    checkListClear(127);
+    checkListClear(128);
+    checkListClear(129);
+  }
+
+  private void checkListClear(int n) {
+    for (int i = 0; i < 4; i++) {
+      ChunkList<String> list = new ChunkList<>(Collections.nCopies(n, "z"));
+      assertThat(list.size(), is(n));
+      switch (i) {
+      case 0:
+        list.clear();
+        break;
+      case 1:
+        for (int j = 0; j < n; j++) {
+          list.remove(0);
+        }
+        break;
+      case 2:
+        for (int j = 0; j < n; j++) {
+          list.remove(list.size() - 1);
+        }
+        break;
+      case 3:
+        Random random = new Random();
+        for (int j = 0; j < n; j++) {
+          list.remove(random.nextInt(list.size()));
+        }
+        break;
+      }
+      assertThat(list.isEmpty(), is(true));
+    }
+  }
+
+  /**
+   * Removing via an iterator.
+   */
+  @Test public void testIterator() {
+    final ChunkList<String> list = new ChunkList<>();
+    list.add("a");
+    list.add("b");
+    final ListIterator<String> listIterator = list.listIterator(0);
+    try {
+      listIterator.remove();
+      fail("excepted exception");
+    } catch (IllegalStateException e) {
+      // ok
+    }
+    listIterator.next();
+    listIterator.remove();
+    assertThat(list.size(), is(1));
+    assertThat(listIterator.hasNext(), is(true));
+    listIterator.next();
+    listIterator.remove();
+    assertThat(list.size(), is(0));
+    assertThat(listIterator.hasNext(), is(false));
   }
 
   /**
@@ -142,41 +264,52 @@ public class ChunkListTest {
    */
   @Test public void testRandom() {
     final int iterationCount = 10000;
-    checkRandom(new Random(1), new ChunkList<Integer>(), iterationCount);
+    checkRandom(new Random(1), new ChunkList<Integer>(),
+        new ArrayList<Integer>(), iterationCount);
     final Random random = new Random(2);
     for (int j = 0; j < 10; j++) {
-      checkRandom(random, new ChunkList<Integer>(), iterationCount);
+      checkRandom(random, new ChunkList<Integer>(), new ArrayList<Integer>(),
+          iterationCount);
     }
-    checkRandom(
-        new Random(3), new ChunkList<Integer>(Collections.nCopies(1000, 5)),
-        iterationCount);
+    final ChunkList<Integer> chunkList =
+        new ChunkList<>(Collections.nCopies(1000, 5));
+    final List<Integer> referenceList = new ArrayList<>(chunkList);
+    checkRandom(new Random(3), chunkList, referenceList, iterationCount);
   }
 
   void checkRandom(
       Random random,
       ChunkList<Integer> list,
+      List<Integer> list2,
       int iterationCount) {
     int removeCount = 0;
     int addCount = 0;
+    int size;
+    int e;
     final int initialCount = list.size();
     for (int i = 0; i < iterationCount; i++) {
       assert list.isValid(true);
-      switch (random.nextInt(8)) {
+      switch (random.nextInt(10)) {
       case 0:
         // remove last
         if (!list.isEmpty()) {
+          assertThat(list2.isEmpty(), is(false));
           list.remove(list.size() - 1);
+          list2.remove(list2.size() - 1);
           ++removeCount;
         }
         break;
       case 1:
         // add to end
-        list.add(random.nextInt(1000));
+        e = random.nextInt(1000);
+        list.add(e);
+        list2.add(e);
         ++addCount;
         break;
       case 2:
         int n = 0;
-        final int size = list.size();
+        size = list.size();
+        assertThat(list.size(), is(list2.size()));
         for (Integer integer : list) {
           Util.discard(integer);
           assertTrue(n++ < size);
@@ -184,20 +317,26 @@ public class ChunkListTest {
         break;
       case 3:
         // remove all instances of a particular value
-        int sizeBefore = list.size();
-        boolean b = list.removeAll(
-            Collections.singletonList(random.nextInt(500)));
+        size = list.size();
+        final List<Integer> zz = Collections.singletonList(random.nextInt(500));
+        boolean b = list.removeAll(zz);
+        boolean b2 = list2.removeAll(zz);
+        assertThat(b, is(b2));
         if (b) {
-          assertTrue(list.size() < sizeBefore);
+          assertTrue(list.size() < size);
+          assertTrue(list2.size() < size);
         } else {
-          assertTrue(list.size() == sizeBefore);
+          assertTrue(list.size() == size);
+          assertTrue(list2.size() == size);
         }
-        removeCount += sizeBefore - list.size();
+        removeCount += size - list.size();
         break;
       case 4:
         // remove at random position
         if (!list.isEmpty()) {
-          list.remove(random.nextInt(list.size()));
+          e = random.nextInt(list.size());
+          list.remove(e);
+          list2.remove(e);
           ++removeCount;
         }
         break;
@@ -205,19 +344,35 @@ public class ChunkListTest {
         // add at random position
         int count = random.nextInt(list.size() + 1);
         ListIterator<Integer> it = list.listIterator();
+        ListIterator<Integer> it2 = list2.listIterator();
         for (int j = 0; j < count; j++) {
           it.next();
+          it2.next();
         }
-        it.add(list.size());
+        size = list.size();
+        it.add(size);
+        it2.add(size);
         ++addCount;
         break;
+      case 6:
+        // clear
+        if (random.nextInt(200) == 0) {
+          removeCount += list.size();
+          list.clear();
+          list2.clear();
+        }
+        break;
       default:
         // add at random position
-        list.add(random.nextInt(list.size() + 1), list.size());
+        int pos = random.nextInt(list.size() + 1);
+        e = list.size();
+        list.add(pos, e);
+        list2.add(pos, e);
         ++addCount;
         break;
       }
       assertEquals(list.size(), initialCount + addCount - removeCount);
+      assertEquals(list, list2);
     }
   }
 
@@ -231,22 +386,22 @@ public class ChunkListTest {
             Arrays.asList(
                 new Function0<List<Integer>>() {
                   public List<Integer> apply() {
-                    return new ArrayList<Integer>();
+                    return new ArrayList<>();
                   }
                 },
                 new Function0<List<Integer>>() {
                   public List<Integer> apply() {
-                    return new LinkedList<Integer>();
+                    return new LinkedList<>();
                   }
                 },
                 new Function0<List<Integer>>() {
                   public List<Integer> apply() {
-                    return new ChunkList<Integer>();
+                    return new ChunkList<>();
                   }
                 }),
             Arrays.asList("ArrayList", "LinkedList", "ChunkList-64"));
     final List<Pair<Function0<List<Integer>>, String>> factories1 =
-        new ArrayList<Pair<Function0<List<Integer>>, String>>();
+        new ArrayList<>();
     for (Pair<Function0<List<Integer>>, String> pair : factories0) {
       factories1.add(pair);
     }


[38/50] [abbrv] calcite git commit: [CALCITE-1051] Underflow exception due to scaling IN clause literals (Frankie Bollaert)

Posted by jh...@apache.org.
[CALCITE-1051] Underflow exception due to scaling IN clause literals (Frankie Bollaert)

Literals in the IN clause value list are scaled during SqlToRel
conversion.  When the type of the literal does not have a scale set,
as happens with a java.lang.Integer, the default value of the
type.scale is chosen, which is Integer.MIN_VALUE.  Scaling to this
value causes an underflow.


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

Branch: refs/heads/branch-release
Commit: b4b04d29cb63ceea123ec41dd3247a0555f9c150
Parents: 361096b
Author: Frankie Bollaert <fr...@ngdata.com>
Authored: Mon Jan 11 11:23:46 2016 +0100
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 12 13:47:17 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/sql/type/SqlTypeUtil.java    |  5 +++++
 .../org/apache/calcite/sql2rel/SqlToRelConverter.java    |  2 +-
 .../src/test/java/org/apache/calcite/test/CsvTest.java   | 11 +++++++++++
 3 files changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/b4b04d29/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 204b7ea..d8a437d 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -414,6 +414,11 @@ public abstract class SqlTypeUtil {
     }
   }
 
+  /** Returns whether a type's scale is set. */
+  public static boolean hasScale(RelDataType type) {
+    return type.getScale() != Integer.MIN_VALUE;
+  }
+
   /**
    * Returns the maximum value of an integral type, as a long value
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/b4b04d29/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 bb37de1..d9189a5 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -1650,7 +1650,7 @@ public class SqlToRelConverter {
 
     Comparable value = literal.getValue();
 
-    if (SqlTypeUtil.isExactNumeric(type)) {
+    if (SqlTypeUtil.isExactNumeric(type) && SqlTypeUtil.hasScale(type)) {
       BigDecimal roundedValue =
           NumberUtil.rescaleBigDecimal(
               (BigDecimal) value,

http://git-wip-us.apache.org/repos/asf/calcite/blob/b4b04d29/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
----------------------------------------------------------------------
diff --git a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
index a048319..650dce1 100644
--- a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
+++ b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
@@ -362,6 +362,17 @@ public class CsvTest {
         "smart", expect("NAME=Alice"));
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1051">[CALCITE-1051]
+   * Underflow exception due to scaling IN clause literals</a>. */
+  @Test public void testInToSemiJoinWithoutCast() throws SQLException {
+    final String sql = "SELECT e.name\n"
+        + "FROM emps AS e\n"
+        + "WHERE e.empno in "
+        + range(130, SqlToRelConverter.IN_SUBQUERY_THRESHOLD);
+    checkSql(sql, "smart", expect("NAME=Alice"));
+  }
+
   private String range(int first, int count) {
     final StringBuilder sb = new StringBuilder();
     for (int i = 0; i < count; i++) {


[46/50] [abbrv] calcite git commit: [CALCITE-1055] SubQueryRemoveRule should create Correlate, not Join, for correlated sub-queries

Posted by jh...@apache.org.
[CALCITE-1055] SubQueryRemoveRule should create Correlate, not Join, for correlated sub-queries


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

Branch: refs/heads/branch-release
Commit: cecef9d3209cbd092a6e9b578bc063a52007b99f
Parents: 898fdfc
Author: maryannxue <ma...@gmail.com>
Authored: Thu Jan 14 13:46:07 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Jan 14 14:05:17 2016 -0800

----------------------------------------------------------------------
 .../calcite/rel/rules/SubQueryRemoveRule.java   |  4 +-
 .../apache/calcite/test/RelOptRulesTest.java    |  8 ++++
 .../org/apache/calcite/test/RelOptRulesTest.xml | 39 ++++++++++++++++++--
 3 files changed, 46 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/cecef9d3/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
index 56b362d..6eec2fc 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
@@ -156,7 +156,7 @@ public abstract class SubQueryRemoveRule extends RelOptRule {
             builder.aggregateCall(SqlStdOperatorTable.SINGLE_VALUE, false, null,
                 null, builder.field(0)));
       }
-      builder.join(JoinRelType.LEFT);
+      builder.join(JoinRelType.LEFT, builder.literal(true), variablesSet);
       return field(builder, inputCount, offset);
 
     case IN:
@@ -247,7 +247,7 @@ public abstract class SubQueryRemoveRule extends RelOptRule {
             builder.aggregateCall(SqlStdOperatorTable.COUNT, false, null, "ck",
                 builder.fields()));
         builder.as("ct");
-        builder.join(JoinRelType.INNER);
+        builder.join(JoinRelType.INNER, builder.literal(true), variablesSet);
         offset += 2;
         builder.push(e.rel);
         break;

http://git-wip-us.apache.org/repos/asf/calcite/blob/cecef9d3/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 6300579..f59b9e7 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -2216,6 +2216,14 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkSubQuery(sql).check();
   }
 
+  @Test public void testExpandWhereComparisonCorrelated() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp as e\n"
+        + "where sal = (\n"
+        + "  select max(sal) from sales.emp e2 where e2.empno = e.empno)";
+    checkSubQuery(sql).check();
+  }
+
 }
 
 // End RelOptRulesTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cecef9d3/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 e45b82c..45ea111 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -1064,6 +1064,39 @@ LogicalValues(tuples=[[]])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testExpandWhereComparisonCorrelated">
+        <Resource name="sql">
+            <![CDATA[select empno
+from sales.emp as e
+where sal = (
+  select max(sal) from sales.emp e2 where e2.empno = e.empno)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[=($5, $SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[MAX($0)])
+  LogicalProject(SAL=[$5])
+    LogicalFilter(condition=[=($0, $cor0.EMPNO)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}))], variablesSet=[[$cor0]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+    LogicalFilter(condition=[=($5, $9)])
+      LogicalCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{0}])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{}], EXPR$0=[MAX($0)])
+          LogicalProject(SAL=[$5])
+            LogicalFilter(condition=[=($0, $cor0.EMPNO)])
+              LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testPushSumConstantThroughUnion">
         <Resource name="sql">
             <![CDATA[select ename, sum(u) from
@@ -4944,9 +4977,9 @@ LogicalFilter(condition=[<($0, 20)])
 })])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
-    </Resource>
-    <Resource name="planAfter">
-      <![CDATA[
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
 LogicalProject(EMPNO=[$0])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
     LogicalJoin(condition=[true], joinType=[inner])


[04/50] [abbrv] calcite git commit: [CALCITE-1042] Ensure that FILTER is BOOLEAN NOT NULL

Posted by jh...@apache.org.
[CALCITE-1042] Ensure that FILTER is BOOLEAN NOT NULL


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

Branch: refs/heads/branch-release
Commit: efb668bb58cdc85e85facba3ec2365549778eee8
Parents: 904c73d
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 4 17:34:58 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/rel/core/Aggregate.java  | 13 ++++--
 .../apache/calcite/runtime/CalciteResource.java |  3 ++
 .../calcite/sql2rel/SqlToRelConverter.java      |  4 ++
 .../org/apache/calcite/tools/RelBuilder.java    |  8 ++++
 .../calcite/runtime/CalciteResource.properties  |  1 +
 .../org/apache/calcite/test/RelBuilderTest.java | 44 ++++++++++++++++++++
 .../java/org/apache/calcite/util/Smalls.java    | 23 ++++++++++
 core/src/test/resources/sql/agg.iq              |  2 +-
 .../linq4j/tree/DeterministicCodeOptimizer.java | 22 +++++-----
 9 files changed, 104 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 3a330d8..1a46ffd 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
@@ -140,12 +140,19 @@ public abstract class Aggregate extends SingleRel {
     assert groupSet.length() <= child.getRowType().getFieldCount();
     for (AggregateCall aggCall : aggCalls) {
       assert typeMatchesInferred(aggCall, true);
-      assert aggCall.filterArg < 0
-          || child.getRowType().getFieldList().get(aggCall.filterArg).getType()
-              .getSqlTypeName() == SqlTypeName.BOOLEAN;
+      Preconditions.checkArgument(aggCall.filterArg < 0
+          || isPredicate(child, aggCall.filterArg),
+          "filter must be BOOLEAN NOT NULL");
     }
   }
 
+  private boolean isPredicate(RelNode input, int index) {
+    final RelDataType type =
+        input.getRowType().getFieldList().get(index).getType();
+    return type.getSqlTypeName() == SqlTypeName.BOOLEAN
+        && !type.isNullable();
+  }
+
   /**
    * Creates an Aggregate by parsing serialized output.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 8c93081..ad3bc4f 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -607,6 +607,9 @@ public interface CalciteResource {
   @BaseMessage("Table ''{0}'' not found")
   ExInst<CalciteException> tableNotFound(String tableName);
 
+  @BaseMessage("FILTER expression must be of type BOOLEAN")
+  ExInst<CalciteException> filterMustBeBoolean();
+
   @BaseMessage("Cannot stream results of a query with no streaming inputs: ''{0}''. At least one input should be convertible to a stream")
   ExInst<SqlValidatorException> cannotStreamResultsForNonStreamingInputs(String inputs);
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 2e87b53..3b402af 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -4506,6 +4506,10 @@ public class SqlToRelConverter {
         if (filter != null) {
           RexNode convertedExpr = bb.convertExpression(filter);
           assert convertedExpr != null;
+          if (convertedExpr.getType().isNullable()) {
+            convertedExpr =
+                rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, convertedExpr);
+          }
           filterArg = lookupOrCreateGroupExpr(convertedExpr);
         }
       } finally {

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 2cd786c..252e703 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -600,6 +600,14 @@ public class RelBuilder {
   /** Creates a call to an aggregate function. */
   public AggCall aggregateCall(SqlAggFunction aggFunction, boolean distinct,
       RexNode filter, String alias, Iterable<? extends RexNode> operands) {
+    if (filter != null) {
+      if (filter.getType().getSqlTypeName() != SqlTypeName.BOOLEAN) {
+        throw Static.RESOURCE.filterMustBeBoolean().ex();
+      }
+      if (filter.getType().isNullable()) {
+        filter = call(SqlStdOperatorTable.IS_TRUE, filter);
+      }
+    }
     return new AggCallImpl(aggFunction, distinct, filter, alias,
         ImmutableList.copyOf(operands));
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 85dcc1d..d050f85 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -198,5 +198,6 @@ ModifiableViewMustBeBasedOnSingleTable=Modifiable view must be based on a single
 MoreThanOneMappedColumn=View is not modifiable. More than one expression maps to column ''{0}'' of base table ''{1}''
 NoValueSuppliedForViewColumn=View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}''
 TableNotFound=Table ''{0}'' not found
+FilterMustBeBoolean=FILTER expression must be of type BOOLEAN
 CannotStreamResultsForNonStreamingInputs=Cannot stream results of a query with no streaming inputs: ''{0}''. At least one input should be convertible to a stream
 # End CalciteResource.properties

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 82757ab..7df8dcf 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
@@ -426,6 +427,49 @@ public class RelBuilderTest {
     assertThat(str(root), is(expected));
   }
 
+  @Test public void testAggregateFilterFails() {
+    // Equivalent SQL:
+    //   SELECT deptno, SUM(sal) FILTER (WHERE comm) AS c
+    //   FROM emp
+    //   GROUP BY deptno
+    try {
+      final RelBuilder builder = RelBuilder.create(config().build());
+      RelNode root =
+          builder.scan("EMP")
+              .aggregate(
+                  builder.groupKey(builder.field("DEPTNO")),
+                  builder.aggregateCall(SqlStdOperatorTable.SUM, false,
+                      builder.field("COMM"), "C", builder.field("SAL")))
+              .build();
+      fail("expected error, got " + root);
+    } catch (CalciteException e) {
+      assertThat(e.getMessage(),
+          is("FILTER expression must be of type BOOLEAN"));
+    }
+  }
+
+  @Test public void testAggregateFilterNullable() {
+    // Equivalent SQL:
+    //   SELECT deptno, SUM(sal) FILTER (WHERE comm < 100) AS c
+    //   FROM emp
+    //   GROUP BY deptno
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("EMP")
+            .aggregate(
+                builder.groupKey(builder.field("DEPTNO")),
+                builder.aggregateCall(SqlStdOperatorTable.SUM, false,
+                    builder.call(SqlStdOperatorTable.LESS_THAN,
+                        builder.field("COMM"), builder.literal(100)), "C",
+                    builder.field("SAL")))
+            .build();
+    final String expected = ""
+        + "LogicalAggregate(group=[{7}], C=[SUM($5) FILTER $8])\n"
+        + "  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], $f8=[IS TRUE(<($6, 100))])\n"
+        + "    LogicalTableScan(table=[[scott, EMP]])\n";
+    assertThat(str(root), is(expected));
+  }
+
   @Test public void testAggregateGroupingKeyOutOfRangeFails() {
     final RelBuilder builder = RelBuilder.create(config().build());
     try {

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 fe04094..62cc064 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -24,6 +24,7 @@ import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.Queryable;
+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;
@@ -369,6 +370,15 @@ public class Smalls {
   }
 
   /** See {@link CountArgs0Function}. */
+  public abstract static class CountArgs1NullableFunction {
+    private CountArgs1NullableFunction() {}
+
+    public static int eval(Short x) {
+      return -1;
+    }
+  }
+
+  /** See {@link CountArgs0Function}. */
   public abstract static class CountArgs2Function {
     private CountArgs2Function() {}
 
@@ -404,12 +414,25 @@ public class Smalls {
   }
 
   /** UDF class that provides user-defined functions for each data type. */
+  @Deterministic
   public static class AllTypesFunction {
     private AllTypesFunction() {}
 
     public static long dateFun(java.sql.Date x) { return x == null ? -1L : x.getTime(); }
     public static long timestampFun(java.sql.Timestamp x) { return x == null ? -1L : x.getTime(); }
     public static long timeFun(java.sql.Time x) { return x == null ? -1L : x.getTime(); }
+
+    public static java.sql.Date toDateFun(int x) { return new java.sql.Date(x); }
+
+    public static java.sql.Date toDateFun(Long x) {
+      return x == null ? null : new java.sql.Date(x);
+    }
+    public static java.sql.Timestamp toTimestampFun(Long x) {
+      return x == null ? null : new java.sql.Timestamp(x);
+    }
+    public static java.sql.Time toTimeFun(Long x) {
+      return x == null ? null : new java.sql.Time(x);
+    }
   }
 
   /** Example of a user-defined aggregate function (UDAF). */

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/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 720b3df..b76cc90 100644
--- a/core/src/test/resources/sql/agg.iq
+++ b/core/src/test/resources/sql/agg.iq
@@ -842,7 +842,7 @@ group by deptno;
 
 !ok
 EnumerableAggregate(group=[{0}], CF=[COUNT() FILTER $1], C=[COUNT()])
-  EnumerableCalc(expr#0..1=[{inputs}], expr#2=['CLERK'], expr#3=[=($t0, $t2)], DEPTNO=[$t1], $f1=[$t3])
+  EnumerableCalc(expr#0..1=[{inputs}], expr#2=['CLERK'], expr#3=[=($t0, $t2)], expr#4=[IS TRUE($t3)], DEPTNO=[$t1], $f1=[$t4])
     EnumerableUnion(all=[true])
       EnumerableCalc(expr#0..7=[{inputs}], expr#8=[20], expr#9=[<($t7, $t8)], JOB=[$t2], DEPTNO=[$t7], $condition=[$t9])
         EnumerableTableScan(table=[[scott, EMP]])

http://git-wip-us.apache.org/repos/asf/calcite/blob/efb668bb/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
index f161c43..e117d28 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/DeterministicCodeOptimizer.java
@@ -19,13 +19,13 @@ package org.apache.calcite.linq4j.tree;
 import org.apache.calcite.linq4j.function.Deterministic;
 import org.apache.calcite.linq4j.function.NonDeterministic;
 
+import com.google.common.collect.ImmutableSet;
+
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.math.BigInteger;
-import java.util.Arrays;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
@@ -46,22 +46,20 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
    * For instance, cast expression will not be factored to a field,
    * but we still need to track its constant status.
    */
-  protected final Map<Expression, Boolean> constants =
-      new IdentityHashMap<Expression, Boolean>();
+  protected final Map<Expression, Boolean> constants = new IdentityHashMap<>();
 
   /**
    * The map that de-duplicates expressions, so the same expressions may reuse
    * the same final static fields.
    */
-  protected final Map<Expression, ParameterExpression> dedup =
-      new HashMap<Expression, ParameterExpression>();
+  protected final Map<Expression, ParameterExpression> dedup = new HashMap<>();
 
   /**
    * The map of all the added final static fields. Allows to identify if the
    * name is occupied or not.
    */
   protected final Map<String, ParameterExpression> fieldsByName =
-      new HashMap<String, ParameterExpression>();
+      new HashMap<>();
 
   // Pre-compiled patterns for generation names for the final static fields
   private static final Pattern NON_ASCII = Pattern.compile("[^0-9a-zA-Z$]+");
@@ -70,10 +68,9 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
       Pattern.compile(Pattern.quote(FIELD_PREFIX));
 
   private static final Set<Class> DETERMINISTIC_CLASSES =
-      new HashSet<Class>(
-          Arrays.<Class>asList(Byte.class, Boolean.class, Short.class,
-              Integer.class, Long.class, BigInteger.class, BigDecimal.class,
-              String.class, Math.class));
+      ImmutableSet.<Class>of(Byte.class, Boolean.class, Short.class,
+          Integer.class, Long.class, BigInteger.class, BigDecimal.class,
+          String.class, Math.class);
 
   /**
    * Creates a child optimizer.
@@ -337,7 +334,8 @@ public class DeterministicCodeOptimizer extends ClassDeclarationFinder {
    */
   protected boolean allMethodsDeterministic(Class klass) {
     return DETERMINISTIC_CLASSES.contains(klass)
-           || klass.isAnnotationPresent(Deterministic.class);
+        || klass.getCanonicalName().equals("org.apache.calcite.avatica.util.DateTimeUtils")
+        || klass.isAnnotationPresent(Deterministic.class);
   }
 
   /**


[20/50] [abbrv] calcite git commit: Add ImmutableBitSet.rebuild()

Posted by jh...@apache.org.
Add ImmutableBitSet.rebuild()


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

Branch: refs/heads/branch-release
Commit: 239babd8fc26bf35be96a77fbfbea379138f3a37
Parents: c104c75
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Jan 6 22:10:45 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:25 2016 -0800

----------------------------------------------------------------------
 .../apache/calcite/util/ImmutableBitSet.java    | 61 +++++++++++++++-----
 .../calcite/util/ImmutableBitSetTest.java       |  7 ++-
 2 files changed, 51 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/239babd8/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java b/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
index cb911f1..a8d6a61 100644
--- a/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
+++ b/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
@@ -650,16 +650,16 @@ public class ImmutableBitSet
 
   /** Returns the union of this immutable bit set with a {@link BitSet}. */
   public ImmutableBitSet union(BitSet other) {
-    return builder(this)
+    return rebuild() // remember "this" and try to re-use later
         .addAll(BitSets.toIter(other))
         .build();
   }
 
   /** Returns the union of this bit set with another. */
   public ImmutableBitSet union(ImmutableBitSet other) {
-    return builder(this)
+    return rebuild() // remember "this" and try to re-use later
         .addAll(other)
-        .build();
+        .build(other); // try to re-use "other"
   }
 
   /** Returns the union of a number of bit sets. */
@@ -677,9 +677,9 @@ public class ImmutableBitSet
    *
    *  @see BitSet#andNot(java.util.BitSet) */
   public ImmutableBitSet except(ImmutableBitSet that) {
-    final Builder builder = builder(this);
+    final Builder builder = rebuild();
     builder.removeAll(that);
-    return builder.build(this);
+    return builder.build();
   }
 
   /** Returns a bit set with all the bits set in both this set and in
@@ -687,9 +687,9 @@ public class ImmutableBitSet
    *
    *  @see BitSet#and */
   public ImmutableBitSet intersect(ImmutableBitSet that) {
-    final Builder builder = builder(this);
+    final Builder builder = rebuild();
     builder.intersect(that);
-    return builder.build(this);
+    return builder.build();
   }
 
   /**
@@ -773,12 +773,20 @@ public class ImmutableBitSet
     return words.length == 0;
   }
 
+  /** Creates an empty Builder. */
   public static Builder builder() {
-    return new Builder();
+    return new Builder(EMPTY_LONGS);
   }
 
+  @Deprecated // to be removed before 2.0
   public static Builder builder(ImmutableBitSet bitSet) {
-    return new Builder(bitSet);
+    return bitSet.rebuild();
+  }
+
+  /** Creates a Builder whose initial contents are the same as this
+   * ImmutableBitSet. */
+  public Builder rebuild() {
+    return new Rebuilder(this);
   }
 
   /** Returns the {@code n}th set bit.
@@ -916,12 +924,8 @@ public class ImmutableBitSet
   public static class Builder {
     private long[] words;
 
-    public Builder(ImmutableBitSet bitSet) {
-      words = bitSet.words.clone();
-    }
-
-    public Builder() {
-      words = EMPTY_LONGS;
+    private Builder(long[] words) {
+      this.words = words;
     }
 
     /** Builds an ImmutableBitSet from the contents of this Builder.
@@ -1086,6 +1090,33 @@ public class ImmutableBitSet
       trim(x);
     }
   }
+
+  /** Refinement of {@link Builder} that remembers its original
+   * {@link org.apache.calcite.util.ImmutableBitSet} and tries to use it
+   * when {@link #build} is called. */
+  private static class Rebuilder extends Builder {
+    private final ImmutableBitSet originalBitSet;
+
+    private Rebuilder(ImmutableBitSet originalBitSet) {
+      super(originalBitSet.words.clone());
+      this.originalBitSet = originalBitSet;
+    }
+
+    @Override public ImmutableBitSet build() {
+      if (wouldEqual(originalBitSet)) {
+        return originalBitSet;
+      }
+      return super.build();
+    }
+
+    @Override public ImmutableBitSet build(ImmutableBitSet bitSet) {
+      // We try to re-use both originalBitSet and bitSet.
+      if (wouldEqual(originalBitSet)) {
+        return originalBitSet;
+      }
+      return super.build(bitSet);
+    }
+  }
 }
 
 // End ImmutableBitSet.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/239babd8/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java b/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java
index a496fd9..bd25a32 100644
--- a/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java
+++ b/core/src/test/java/org/apache/calcite/util/ImmutableBitSetTest.java
@@ -293,11 +293,14 @@ public class ImmutableBitSetTest {
    * {@link org.apache.calcite.util.ImmutableBitSet.Builder#build(ImmutableBitSet)}. */
   @Test public void testBuilderUseOriginal() {
     final ImmutableBitSet fives = ImmutableBitSet.of(5, 10, 15);
+    final ImmutableBitSet fives1 =
+        fives.rebuild().clear(2).set(10).build();
+    assertTrue(fives1 == fives);
     final ImmutableBitSet fives2 =
-        ImmutableBitSet.builder(fives).clear(2).set(10).build(fives);
+        ImmutableBitSet.builder().addAll(fives).clear(2).set(10).build(fives);
     assertTrue(fives2 == fives);
     final ImmutableBitSet fives3 =
-        ImmutableBitSet.builder(fives).clear(2).set(10).build();
+        ImmutableBitSet.builder().addAll(fives).clear(2).set(10).build();
     assertTrue(fives3 != fives);
     assertTrue(fives3.equals(fives));
     assertTrue(fives3.equals(fives2));


[19/50] [abbrv] calcite git commit: [CALCITE-915] Tests should unset ThreadLocal values on exit

Posted by jh...@apache.org.
[CALCITE-915] Tests should unset ThreadLocal values on exit


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

Branch: refs/heads/branch-release
Commit: c104c75bdf6d663070cf16b5ab0f94f8c269c9b6
Parents: 7837e64
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Oct 8 16:23:20 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/prepare/Prepare.java     |  15 +-
 .../org/apache/calcite/util/TryThreadLocal.java |  71 ++++++++
 .../java/org/apache/calcite/test/JdbcTest.java  | 180 +++++++++----------
 .../calcite/test/MaterializationTest.java       |  21 +--
 .../java/org/apache/calcite/util/UtilTest.java  |  37 ++++
 5 files changed, 212 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/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 afb722f..3530e93 100644
--- a/core/src/main/java/org/apache/calcite/prepare/Prepare.java
+++ b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
@@ -52,6 +52,7 @@ import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.Programs;
 import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.TryThreadLocal;
 import org.apache.calcite.util.trace.CalciteTimingTracer;
 import org.apache.calcite.util.trace.CalciteTrace;
 
@@ -83,12 +84,14 @@ public abstract class Prepare {
   protected RelDataType parameterRowType;
 
   // temporary. for testing.
-  public static final ThreadLocal<Boolean> THREAD_TRIM =
-      new ThreadLocal<Boolean>() {
-        @Override protected Boolean initialValue() {
-          return false;
-        }
-      };
+  public static final TryThreadLocal<Boolean> THREAD_TRIM =
+      TryThreadLocal.of(false);
+
+  /** Temporary, while CALCITE-816 is under development.
+   *
+   * @see org.apache.calcite.util.Util#deprecated(Object, boolean) */
+  public static final TryThreadLocal<Boolean> THREAD_EXPAND =
+      TryThreadLocal.of(false);
 
   public Prepare(CalcitePrepare.Context context, CatalogReader catalogReader,
       Convention resultConvention) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/core/src/main/java/org/apache/calcite/util/TryThreadLocal.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/TryThreadLocal.java b/core/src/main/java/org/apache/calcite/util/TryThreadLocal.java
new file mode 100644
index 0000000..b278174
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/TryThreadLocal.java
@@ -0,0 +1,71 @@
+/*
+ * 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.util;
+
+/**
+ * Thread-local variable that returns a handle that can be closed.
+ *
+ * @param <T> Value type
+ */
+public class TryThreadLocal<T> extends ThreadLocal<T> {
+  private final T initialValue;
+
+  /** Creates a TryThreadLocal.
+   *
+   * @param initialValue Initial value
+   */
+  public static <T> TryThreadLocal<T> of(T initialValue) {
+    return new TryThreadLocal<>(initialValue);
+  }
+
+  private TryThreadLocal(T initialValue) {
+    this.initialValue = initialValue;
+  }
+
+  // It is important that this method is final.
+  // This ensures that the sub-class does not choose a different initial
+  // value. Then the close logic can detect whether the previous value was
+  // equal to the initial value.
+  @Override protected final T initialValue() {
+    return initialValue;
+  }
+
+  /** Assigns the value as {@code value} for the current thread.
+   * Returns a {@link Memo} which, when closed, will assign the value
+   * back to the previous value. */
+  public Memo push(T value) {
+    final T previous = get();
+    set(value);
+    return new Memo() {
+      public void close() {
+        if (previous == initialValue) {
+          remove();
+        } else {
+          set(previous);
+        }
+      }
+    };
+  }
+
+  /** Remembers to set the value back. */
+  public interface Memo extends AutoCloseable {
+    /** Sets the value back; never throws. */
+    @Override void close();
+  }
+}
+
+// End TryThreadLocal.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/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 b64cba4..243f903 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -91,6 +91,7 @@ import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.JsonBuilder;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Smalls;
+import org.apache.calcite.util.TryThreadLocal;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -247,8 +248,8 @@ public class JdbcTest {
   @Test public void testModelWithModifiableView() throws Exception {
     final List<Employee> employees = new ArrayList<>();
     employees.add(new Employee(135, 10, "Simon", 56.7f, null));
-    try {
-      EmpDeptTableFactory.THREAD_COLLECTION.set(employees);
+    try (final TryThreadLocal.Memo ignore =
+             EmpDeptTableFactory.THREAD_COLLECTION.push(employees)) {
       final CalciteAssert.AssertThat with = modelWithView(
           "select \"name\", \"empid\" as e, \"salary\" "
               + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" = 10",
@@ -316,8 +317,6 @@ public class JdbcTest {
               }
             }
           });
-    } finally {
-      EmpDeptTableFactory.THREAD_COLLECTION.remove();
     }
   }
 
@@ -325,9 +324,8 @@ public class JdbcTest {
   @Test public void testModelWithInvalidModifiableView() throws Exception {
     final List<Employee> employees = new ArrayList<>();
     employees.add(new Employee(135, 10, "Simon", 56.7f, null));
-    try {
-      EmpDeptTableFactory.THREAD_COLLECTION.set(employees);
-
+    try (final TryThreadLocal.Memo ignore =
+             EmpDeptTableFactory.THREAD_COLLECTION.push(employees)) {
       Util.discard(RESOURCE.noValueSuppliedForViewColumn(null, null));
       modelWithView("select \"name\", \"empid\" as e, \"salary\" "
               + "from \"MUTABLE_EMPLOYEES\" where \"commission\" = 10",
@@ -397,8 +395,6 @@ public class JdbcTest {
           null)
           .query("select \"name\" from \"adhoc\".V order by \"name\"")
           .runs();
-    } finally {
-      EmpDeptTableFactory.THREAD_COLLECTION.remove();
     }
   }
 
@@ -871,67 +867,67 @@ public class JdbcTest {
   @Test public void testOnConnectionClose() throws Exception {
     final int[] closeCount = {0};
     final int[] statementCloseCount = {0};
-    HandlerDriver.HANDLERS.set(
-        new HandlerImpl() {
-          @Override public void
-          onConnectionClose(AvaticaConnection connection) {
-            ++closeCount[0];
-            throw new RuntimeException();
-          }
-          @Override public void onStatementClose(AvaticaStatement statement) {
-            ++statementCloseCount[0];
-            throw new RuntimeException();
-          }
-        });
-    final HandlerDriver driver =
-        new HandlerDriver();
-    CalciteConnection connection = (CalciteConnection)
-        driver.connect("jdbc:calcite:", new Properties());
-    SchemaPlus rootSchema = connection.getRootSchema();
-    rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
-    connection.setSchema("hr");
-    final Statement statement = connection.createStatement();
-    final ResultSet resultSet =
-        statement.executeQuery("select * from \"emps\"");
-    assertEquals(0, closeCount[0]);
-    assertEquals(0, statementCloseCount[0]);
-    resultSet.close();
-    try {
-      resultSet.next();
-      fail("resultSet.next() should throw SQLException when closed");
-    } catch (SQLException e) {
-      assertThat(e.getMessage(),
-          containsString("next() called on closed cursor"));
-    }
-    assertEquals(0, closeCount[0]);
-    assertEquals(0, statementCloseCount[0]);
+    final HandlerImpl h = new HandlerImpl() {
+      @Override public void onConnectionClose(AvaticaConnection connection) {
+        ++closeCount[0];
+        throw new RuntimeException();
+      }
 
-    // Close statement. It throws SQLException, but statement is still closed.
-    try {
-      statement.close();
-      fail("expecting error");
-    } catch (SQLException e) {
-      // ok
-    }
-    assertEquals(0, closeCount[0]);
-    assertEquals(1, statementCloseCount[0]);
+      @Override public void onStatementClose(AvaticaStatement statement) {
+        ++statementCloseCount[0];
+        throw new RuntimeException();
+      }
+    };
+    try (final TryThreadLocal.Memo ignore =
+             HandlerDriver.HANDLERS.push(h)) {
+      final HandlerDriver driver = new HandlerDriver();
+      CalciteConnection connection = (CalciteConnection)
+          driver.connect("jdbc:calcite:", new Properties());
+      SchemaPlus rootSchema = connection.getRootSchema();
+      rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
+      connection.setSchema("hr");
+      final Statement statement = connection.createStatement();
+      final ResultSet resultSet =
+          statement.executeQuery("select * from \"emps\"");
+      assertEquals(0, closeCount[0]);
+      assertEquals(0, statementCloseCount[0]);
+      resultSet.close();
+      try {
+        resultSet.next();
+        fail("resultSet.next() should throw SQLException when closed");
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            containsString("next() called on closed cursor"));
+      }
+      assertEquals(0, closeCount[0]);
+      assertEquals(0, statementCloseCount[0]);
 
-    // Close connection. It throws SQLException, but connection is still closed.
-    try {
-      connection.close();
-      fail("expecting error");
-    } catch (SQLException e) {
-      // ok
-    }
-    assertEquals(1, closeCount[0]);
-    assertEquals(1, statementCloseCount[0]);
+      // Close statement. It throws SQLException, but statement is still closed.
+      try {
+        statement.close();
+        fail("expecting error");
+      } catch (SQLException e) {
+        // ok
+      }
+      assertEquals(0, closeCount[0]);
+      assertEquals(1, statementCloseCount[0]);
 
-    // Close a closed connection. Handler is not called again.
-    connection.close();
-    assertEquals(1, closeCount[0]);
-    assertEquals(1, statementCloseCount[0]);
+      // Close connection. It throws SQLException, but connection is still closed.
+      try {
+        connection.close();
+        fail("expecting error");
+      } catch (SQLException e) {
+        // ok
+      }
+      assertEquals(1, closeCount[0]);
+      assertEquals(1, statementCloseCount[0]);
+
+      // Close a closed connection. Handler is not called again.
+      connection.close();
+      assertEquals(1, closeCount[0]);
+      assertEquals(1, statementCloseCount[0]);
 
-    HandlerDriver.HANDLERS.remove();
+    }
   }
 
   /** Tests {@link java.sql.Statement}.{@code closeOnCompletion()}. */
@@ -3335,15 +3331,18 @@ public class JdbcTest {
 
   /** Query that reads no columns from either underlying table. */
   @Test public void testCountStar() {
-    CalciteAssert.hr()
-        .query("select count(*) c from \"hr\".\"emps\", \"hr\".\"depts\"")
-        .convertContains("LogicalAggregate(group=[{}], C=[COUNT()])\n"
-            + "  LogicalProject(DUMMY=[0])\n"
-            + "    LogicalJoin(condition=[true], joinType=[inner])\n"
-            + "      LogicalProject(DUMMY=[0])\n"
-            + "        EnumerableTableScan(table=[[hr, emps]])\n"
-            + "      LogicalProject(DUMMY=[0])\n"
-            + "        EnumerableTableScan(table=[[hr, depts]])");
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true);
+         final TryThreadLocal.Memo memo = Prepare.THREAD_EXPAND.push(true)) {
+      CalciteAssert.hr()
+          .query("select count(*) c from \"hr\".\"emps\", \"hr\".\"depts\"")
+          .convertContains("LogicalAggregate(group=[{}], C=[COUNT()])\n"
+              + "  LogicalProject(DUMMY=[0])\n"
+              + "    LogicalJoin(condition=[true], joinType=[inner])\n"
+              + "      LogicalProject(DUMMY=[0])\n"
+              + "        EnumerableTableScan(table=[[hr, emps]])\n"
+              + "      LogicalProject(DUMMY=[0])\n"
+              + "        EnumerableTableScan(table=[[hr, depts]])");
+    }
   }
 
   /** Same result (and plan) as {@link #testSelectDistinct}. */
@@ -4173,26 +4172,25 @@ public class JdbcTest {
 
   /** Tests that field-trimming creates a project near the table scan. */
   @Test public void testTrimFields() throws Exception {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       CalciteAssert.hr()
           .query("select \"name\", count(\"commission\") + 1\n"
-            + "from \"hr\".\"emps\"\n"
-            + "group by \"deptno\", \"name\"")
+              + "from \"hr\".\"emps\"\n"
+              + "group by \"deptno\", \"name\"")
           .convertContains("LogicalProject(name=[$1], EXPR$1=[+($2, 1)])\n"
               + "  LogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)])\n"
               + "    LogicalProject(deptno=[$1], name=[$2], commission=[$4])\n"
               + "      EnumerableTableScan(table=[[hr, emps]])\n");
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
   /** Tests that field-trimming creates a project near the table scan, in a
    * query with windowed-aggregation. */
   @Test public void testTrimFieldsOver() throws Exception {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true);
+         final TryThreadLocal.Memo memo = Prepare.THREAD_EXPAND.push(true)) {
+      Util.discard(memo);
+      // The correct plan has a project on a filter on a project on a scan.
       CalciteAssert.hr()
           .query("select \"name\",\n"
               + "  count(\"commission\") over (partition by \"deptno\") + 1\n"
@@ -4203,8 +4201,6 @@ public class JdbcTest {
               + "  LogicalFilter(condition=[>($0, 10)])\n"
               + "    LogicalProject(empid=[$0], deptno=[$1], name=[$2], commission=[$4])\n"
               + "      EnumerableTableScan(table=[[hr, emps]])\n");
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
@@ -4220,9 +4216,10 @@ public class JdbcTest {
             "M=1",
             "M=1");
   }
+
   /** Tests multiple window aggregates over constants.
    * This tests that EnumerableWindowRel is able to reference the right slot
-   * when accessing constant for aggregation argument.*/
+   * when accessing constant for aggregation argument. */
   @Test public void testWinAggConstantMultipleConstants() {
     CalciteAssert.that()
         .with(CalciteAssert.Config.REGULAR)
@@ -4644,7 +4641,9 @@ public class JdbcTest {
    * use as scratch space during development. */
   // Do not add '@Ignore'; just remember not to commit changes to dummy.iq
   @Test public void testRunDummy() throws Exception {
-    checkRun("sql/dummy.iq");
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(false)) {
+      checkRun("sql/dummy.iq");
+    }
   }
 
   @Test public void testRunAgg() throws Exception {
@@ -6478,8 +6477,8 @@ public class JdbcTest {
 
   /** Factory for EMP and DEPT tables. */
   public static class EmpDeptTableFactory implements TableFactory<Table> {
-    public static final ThreadLocal<List<Employee>> THREAD_COLLECTION =
-        new ThreadLocal<>();
+    public static final TryThreadLocal<List<Employee>> THREAD_COLLECTION =
+        TryThreadLocal.of(null);
 
     public Table create(
         SchemaPlus schema,
@@ -6578,7 +6577,8 @@ public class JdbcTest {
 
   /** Mock driver that a given {@link Handler}. */
   public static class HandlerDriver extends org.apache.calcite.jdbc.Driver {
-    private static final ThreadLocal<Handler> HANDLERS = new ThreadLocal<>();
+    private static final TryThreadLocal<Handler> HANDLERS =
+        TryThreadLocal.of(null);
 
     public HandlerDriver() {
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/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 65aabd7..05726b3 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -34,6 +34,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.JsonBuilder;
+import org.apache.calcite.util.TryThreadLocal;
 import org.apache.calcite.util.Util;
 
 import org.apache.commons.lang3.StringUtils;
@@ -127,8 +128,7 @@ public class MaterializationTest {
   }
 
   @Test public void testFilterQueryOnProjectView() {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(
@@ -140,8 +140,6 @@ public class MaterializationTest {
           .enableMaterializations(true)
           .explainContains("EnumerableTableScan(table=[[hr, m0]])")
           .sameResultWithMaterializationsDisabled();
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
@@ -155,8 +153,7 @@ public class MaterializationTest {
    * definition. */
   private void checkMaterialize(String materialize, String query, String model,
       Function<ResultSet, Void> explainChecker) {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(model, "m0", materialize)
@@ -164,8 +161,6 @@ public class MaterializationTest {
           .enableMaterializations(true)
           .explainMatches("", explainChecker)
           .sameResultWithMaterializationsDisabled();
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
@@ -173,16 +168,13 @@ public class MaterializationTest {
    * definition. */
   private void checkNoMaterialize(String materialize, String query,
       String model) {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(model, "m0", materialize)
           .query(query)
           .enableMaterializations(true)
           .explainContains("EnumerableTableScan(table=[[hr, emps]])");
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
@@ -876,8 +868,7 @@ public class MaterializationTest {
    * Pre-populated materializations</a>. */
   @Test public void testPrePopulated() {
     String q = "select \"deptno\" from \"emps\"";
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(
@@ -897,8 +888,6 @@ public class MaterializationTest {
           .enableMaterializations(true)
           .explainMatches("", CONTAINS_LOCATIONS)
           .sameResultWithMaterializationsDisabled();
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 78202e2..60122fb 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -73,6 +73,7 @@ import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.isA;
 import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.nullValue;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
@@ -1472,6 +1473,42 @@ public class UtilTest {
     final String asString = Util.listToString(list);
     assertThat(Util.stringToList(asString), is(list));
   }
+
+  /** Tests {@link org.apache.calcite.util.TryThreadLocal}.
+   *
+   * <p>TryThreadLocal was introduced to fix
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-915">[CALCITE-915]
+   * Tests do not unset ThreadLocal values on exit</a>. */
+  @Test public void testTryThreadLocal() {
+    final TryThreadLocal<String> local1 = TryThreadLocal.of("foo");
+    assertThat(local1.get(), is("foo"));
+    TryThreadLocal.Memo memo1 = local1.push("bar");
+    assertThat(local1.get(), is("bar"));
+    local1.set("baz");
+    assertThat(local1.get(), is("baz"));
+    memo1.close();
+    assertThat(local1.get(), is("foo"));
+
+    final TryThreadLocal<String> local2 = TryThreadLocal.of(null);
+    assertThat(local2.get(), nullValue());
+    TryThreadLocal.Memo memo2 = local2.push("a");
+    assertThat(local2.get(), is("a"));
+    local2.set("b");
+    assertThat(local2.get(), is("b"));
+    TryThreadLocal.Memo memo2B = local2.push(null);
+    assertThat(local2.get(), nullValue());
+    memo2B.close();
+    assertThat(local2.get(), is("b"));
+    memo2.close();
+    assertThat(local2.get(), nullValue());
+
+    local2.set("x");
+    try (TryThreadLocal.Memo ignore = local2.push("y")) {
+      assertThat(local2.get(), is("y"));
+      local2.set("z");
+    }
+    assertThat(local2.get(), is("x"));
+  }
 }
 
 // End UtilTest.java


[34/50] [abbrv] calcite git commit: [CALCITE-842] Decorrelator gets field offsets confused if fields have been trimmed

Posted by jh...@apache.org.
[CALCITE-842] Decorrelator gets field offsets confused if fields have been trimmed


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

Branch: refs/heads/branch-release
Commit: 4b519b9882c861bf366e2c9d9928cd6deb5cc8b9
Parents: 505a906
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Jan 7 00:06:19 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:25 2016 -0800

----------------------------------------------------------------------
 .../apache/calcite/rex/RexCorrelVariable.java   |   4 +-
 .../sql2rel/CorrelationReferenceFinder.java     |  74 +++++++++++
 .../apache/calcite/sql2rel/RelDecorrelator.java |  33 ++---
 .../apache/calcite/sql2rel/RelFieldTrimmer.java | 133 ++++++++++++-------
 .../calcite/sql2rel/SqlToRelConverter.java      |  20 +--
 .../apache/calcite/test/JdbcAdapterTest.java    |  93 ++++++-------
 .../java/org/apache/calcite/test/JdbcTest.java  |   8 +-
 .../org/apache/calcite/test/LatticeTest.java    |   2 +-
 .../apache/calcite/test/SqlToRelTestBase.java   |   2 +-
 .../enumerable/EnumerableCorrelateTest.java     |   5 +-
 core/src/test/resources/sql/subquery.iq         |   6 +-
 11 files changed, 236 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java b/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
index 4880c8e..2f6197a 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
@@ -20,6 +20,8 @@ import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Reference to the current row of a correlating relational expression.
  *
@@ -36,7 +38,7 @@ public class RexCorrelVariable extends RexVariable {
       CorrelationId id,
       RelDataType type) {
     super(id.getName(), type);
-    this.id = id;
+    this.id = Preconditions.checkNotNull(id);
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/core/src/main/java/org/apache/calcite/sql2rel/CorrelationReferenceFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/CorrelationReferenceFinder.java b/core/src/main/java/org/apache/calcite/sql2rel/CorrelationReferenceFinder.java
new file mode 100644
index 0000000..db84b4a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql2rel/CorrelationReferenceFinder.java
@@ -0,0 +1,74 @@
+/*
+ * 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.sql2rel;
+
+import org.apache.calcite.rel.RelHomogeneousShuttle;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
+
+/**
+ * Shuttle that finds references to a given {@link CorrelationId} within a tree
+ * of {@link RelNode}s.
+ */
+public abstract class CorrelationReferenceFinder extends RelHomogeneousShuttle {
+  private final MyRexVisitor rexVisitor;
+
+  /** Creates CorrelationReferenceFinder. */
+  protected CorrelationReferenceFinder() {
+    rexVisitor = new MyRexVisitor(this);
+  }
+
+  protected abstract RexNode handle(RexFieldAccess fieldAccess);
+
+  @Override public RelNode visit(RelNode other) {
+    RelNode next = super.visit(other);
+    return next.accept(rexVisitor);
+  }
+
+  /**
+   * Replaces alternative names of correlation variable to its canonical name.
+   */
+  private static class MyRexVisitor extends RexShuttle {
+    private final CorrelationReferenceFinder finder;
+
+    private MyRexVisitor(CorrelationReferenceFinder finder) {
+      this.finder = finder;
+    }
+
+    @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+      if (fieldAccess.getReferenceExpr() instanceof RexCorrelVariable) {
+        return finder.handle(fieldAccess);
+      }
+      return super.visitFieldAccess(fieldAccess);
+    }
+
+    @Override public RexNode visitSubQuery(RexSubQuery subQuery) {
+      final RelNode r = subQuery.rel.accept(finder); // look inside sub-queries
+      if (r != subQuery.rel) {
+        subQuery = subQuery.clone(r);
+      }
+      return super.visitSubQuery(subQuery);
+    }
+  }
+}
+
+// End CorrelationReferenceFinder.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/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 2812851..e1d9f93 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -701,8 +701,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     final Map<RelNode, Integer> mapNewInputToNewOffset = new HashMap<>();
 
-    // inputRel provides the definition of a correlated variable.
-    // Add to map all the referenced positions(relative to each input rel)
+    // Input provides the definition of a correlated variable.
+    // Add to map all the referenced positions (relative to each input rel).
     for (Correlation corVar : correlations) {
       final int oldCorVarOffset = corVar.field;
 
@@ -2365,14 +2365,15 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     public int compareTo(Correlation o) {
-      int res = corr.compareTo(o.corr);
-      if (res != 0) {
-        return res;
+      int c = corr.compareTo(o.corr);
+      if (c != 0) {
+        return c;
       }
-      if (field != o.field) {
-        return field - o.field;
+      c = Integer.compare(field, o.field);
+      if (c != 0) {
+        return c;
       }
-      return uniqueKey - o.uniqueKey;
+      return Integer.compare(uniqueKey, o.uniqueKey);
     }
   }
 
@@ -2545,26 +2546,10 @@ public class RelDecorrelator implements ReflectiveVisitor {
                     corrIdGenerator++);
             mapFieldAccessToCorVar.put(fieldAccess, correlation);
             mapRefRelToCorVar.put(rel, correlation);
-/*
-            if (!mapCorVarToCorRel.containsKey(var.id)) {
-              mapCorVarToCorRel.put(var.id, Stacks.peek(stack));
-            }
-*/
           }
           return super.visitFieldAccess(fieldAccess);
         }
 
-        //@ Override
-        public Void visitCorrelVariable_(RexCorrelVariable var) {
-          final Correlation correlation =
-              new Correlation(var.id, -1, corrIdGenerator++);
-          mapRefRelToCorVar.put(rel, correlation);
-          if (!mapCorVarToCorRel.containsKey(var.id)) {
-            mapCorVarToCorRel.put(var.id, Stacks.peek(stack));
-          }
-          return super.visitCorrelVariable(var);
-        }
-
         @Override public Void visitSubQuery(RexSubQuery subQuery) {
           subQuery.rel.accept(CorelMapBuilder.this);
           return super.visitSubQuery(subQuery);

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/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 2294c4a..8638df0 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -24,6 +24,7 @@ import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.Project;
@@ -37,9 +38,12 @@ import org.apache.calcite.rel.logical.LogicalTableModify;
 import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPermuteInputsShuttle;
@@ -66,8 +70,10 @@ import com.google.common.collect.Iterables;
 import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.logging.Level;
 
@@ -101,6 +107,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
   private final ReflectUtil.MethodDispatcher<TrimResult> trimFieldsDispatcher;
   private final RelBuilder relBuilder;
+  private Map<RelNode, Mapping> map = new HashMap<>();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -177,23 +184,36 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
   protected TrimResult trimChild(
       RelNode rel,
       RelNode input,
-      ImmutableBitSet fieldsUsed,
+      final ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
-    Util.discard(rel);
-    if (input.getClass().getName().endsWith("MedMdrClassExtentRel")) {
-      // MedMdrJoinRule cannot handle Join of Project of
-      // MedMdrClassExtentRel, only naked MedMdrClassExtentRel.
-      // So, disable trimming.
-      fieldsUsed = ImmutableBitSet.range(input.getRowType().getFieldCount());
-    }
+    final ImmutableBitSet.Builder fieldsUsedBuilder = fieldsUsed.rebuild();
+
+    // Fields that define the collation cannot be discarded.
     final ImmutableList<RelCollation> collations =
         RelMetadataQuery.collations(input);
     for (RelCollation collation : collations) {
       for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
-        fieldsUsed = fieldsUsed.set(fieldCollation.getFieldIndex());
+        fieldsUsedBuilder.set(fieldCollation.getFieldIndex());
       }
     }
-    return dispatchTrimFields(input, fieldsUsed, extraFields);
+
+    // Correlating variables are a means for other relational expressions to use
+    // fields.
+    for (final CorrelationId correlation : rel.getVariablesSet()) {
+      rel.accept(
+          new CorrelationReferenceFinder() {
+            protected RexNode handle(RexFieldAccess fieldAccess) {
+              final RexCorrelVariable v =
+                  (RexCorrelVariable) fieldAccess.getReferenceExpr();
+              if (v.id.equals(correlation)) {
+                fieldsUsedBuilder.set(fieldAccess.getField().getIndex());
+              }
+              return fieldAccess;
+            }
+          });
+    }
+
+    return dispatchTrimFields(input, fieldsUsedBuilder.build(), extraFields);
   }
 
   /**
@@ -234,8 +254,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     }
     relBuilder.push(trimResult.left)
         .project(exprList, nameList);
-    return new TrimResult(
-        relBuilder.build(),
+    return result(relBuilder.build(),
         Mappings.createIdentity(fieldList.size()));
   }
 
@@ -268,11 +287,44 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       assert newFieldCount > 0 : "rel has no fields after trim: " + rel;
     }
     if (newRel.equals(rel)) {
-      return new TrimResult(rel, mapping);
+      return result(rel, mapping);
     }
     return trimResult;
   }
 
+  private TrimResult result(RelNode r, final Mapping mapping) {
+    map.put(r, mapping);
+    final RexBuilder rexBuilder = relBuilder.getRexBuilder();
+    final RelNode r0 = r;
+    for (final CorrelationId correlation : r.getVariablesSet()) {
+      r = r.accept(
+          new CorrelationReferenceFinder() {
+            protected RexNode handle(RexFieldAccess fieldAccess) {
+              final RexCorrelVariable v =
+                  (RexCorrelVariable) fieldAccess.getReferenceExpr();
+              if (v.id.equals(correlation)
+                  && v.getType().getFieldCount() == mapping.getSourceCount()) {
+                final int old = fieldAccess.getField().getIndex();
+                final int new_ = mapping.getTarget(old);
+                final RelDataTypeFactory.FieldInfoBuilder typeBuilder =
+                    relBuilder.getTypeFactory().builder();
+                for (int target : Util.range(mapping.getTargetCount())) {
+                  typeBuilder.add(
+                      v.getType().getFieldList().get(mapping.getSource(target)));
+                }
+                final RexNode newV =
+                    rexBuilder.makeCorrel(typeBuilder.build(), v.id);
+                if (old != new_) {
+                  return rexBuilder.makeFieldAccess(newV, new_);
+                }
+              }
+              return fieldAccess;
+            }
+          });
+    }
+    return new TrimResult(r, mapping);
+  }
+
   /**
    * Visit method, per {@link org.apache.calcite.util.ReflectiveVisitor}.
    *
@@ -295,10 +347,8 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // We don't know how to trim this kind of relational expression, so give
     // it back intact.
     Util.discard(fieldsUsed);
-    return new TrimResult(
-        rel,
-        Mappings.createIdentity(
-            rel.getRowType().getFieldCount()));
+    return result(rel,
+        Mappings.createIdentity(rel.getRowType().getFieldCount()));
   }
 
   /**
@@ -336,9 +386,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // there's nothing we can do.
     if (newInput == input
         && fieldsUsed.cardinality() == fieldCount) {
-      return new TrimResult(
-          project,
-          Mappings.createIdentity(fieldCount));
+      return result(project, Mappings.createIdentity(fieldCount));
     }
 
     // Some parts of the system can't handle rows with zero fields, so
@@ -371,7 +419,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     relBuilder.push(newInput);
     relBuilder.project(newProjects, newRowType.getFieldNames());
-    return new TrimResult(relBuilder.build(), mapping);
+    return result(relBuilder.build(), mapping);
   }
 
   /** Creates a project with a dummy column, to protect the parts of the system
@@ -388,13 +436,13 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     if (input.getRowType().getFieldCount() == 1) {
       // Input already has one field (and may in fact be a dummy project we
       // created for the child). We can't do better.
-      return new TrimResult(input, mapping);
+      return result(input, mapping);
     }
     final RexLiteral expr =
         cluster.getRexBuilder().makeExactLiteral(BigDecimal.ZERO);
     relBuilder.push(input);
     relBuilder.project(ImmutableList.<RexNode>of(expr), ImmutableList.of("DUMMY"));
-    return new TrimResult(relBuilder.build(), mapping);
+    return result(relBuilder.build(), mapping);
   }
 
   /**
@@ -430,9 +478,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // there's nothing we can do.
     if (newInput == input
         && fieldsUsed.cardinality() == fieldCount) {
-      return new TrimResult(
-          filter,
-          Mappings.createIdentity(fieldCount));
+      return result(filter, Mappings.createIdentity(fieldCount));
     }
 
     // Build new project expressions, and populate the mapping.
@@ -448,7 +494,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // The result has the same mapping as the input gave us. Sometimes we
     // return fields that the consumer didn't ask for, because the filter
     // needs them for its condition.
-    return new TrimResult(relBuilder.build(), inputMapping);
+    return result(relBuilder.build(), inputMapping);
   }
 
   /**
@@ -484,9 +530,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     if (newInput == input
         && inputMapping.isIdentity()
         && fieldsUsed.cardinality() == fieldCount) {
-      return new TrimResult(
-          sort,
-          Mappings.createIdentity(fieldCount));
+      return result(sort, Mappings.createIdentity(fieldCount));
     }
 
     relBuilder.push(newInput);
@@ -501,7 +545,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // The result has the same mapping as the input gave us. Sometimes we
     // return fields that the consumer didn't ask for, because the filter
     // needs them for its condition.
-    return new TrimResult(relBuilder.build(), inputMapping);
+    return result(relBuilder.build(), inputMapping);
   }
 
   /**
@@ -609,7 +653,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     if (changeCount == 0
         && mapping.isIdentity()) {
-      return new TrimResult(join, Mappings.createIdentity(fieldCount));
+      return result(join, Mappings.createIdentity(fieldCount));
     }
 
     // Build new join.
@@ -641,7 +685,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       relBuilder.join(join.getJoinType(), newConditionExpr);
     }
 
-    return new TrimResult(relBuilder.build(), mapping);
+    return result(relBuilder.build(), mapping);
   }
 
   /**
@@ -701,9 +745,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       for (RelNode input : setOp.getInputs()) {
         relBuilder.build();
       }
-      return new TrimResult(
-          setOp,
-          mapping);
+      return result(setOp, mapping);
     }
 
     switch (setOp.kind) {
@@ -720,7 +762,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     default:
       throw new AssertionError("unknown setOp " + setOp);
     }
-    return new TrimResult(relBuilder.build(), mapping);
+    return result(relBuilder.build(), mapping);
   }
 
   /**
@@ -779,8 +821,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // there's nothing to do.
     if (input == newInput
         && fieldsUsed.equals(ImmutableBitSet.range(rowType.getFieldCount()))) {
-      return new TrimResult(
-          aggregate,
+      return result(aggregate,
           Mappings.createIdentity(rowType.getFieldCount()));
     }
 
@@ -842,7 +883,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
         aggregate.indicator, newGroupSets);
     relBuilder.aggregate(groupKey, newAggCallList);
 
-    return new TrimResult(relBuilder.build(), mapping);
+    return result(relBuilder.build(), mapping);
   }
 
   /**
@@ -889,7 +930,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     // Always project all fields.
     Mapping mapping = Mappings.createIdentity(fieldCount);
-    return new TrimResult(newModifier, mapping);
+    return result(newModifier, mapping);
   }
 
   /**
@@ -928,7 +969,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
 
     // Always project all fields.
     Mapping mapping = Mappings.createIdentity(fieldCount);
-    return new TrimResult(newTabFun, mapping);
+    return result(newTabFun, mapping);
   }
 
   /**
@@ -952,7 +993,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     // If all fields are used, return unchanged.
     if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))) {
       Mapping mapping = Mappings.createIdentity(fieldCount);
-      return new TrimResult(values, mapping);
+      return result(values, mapping);
     }
 
     final ImmutableList.Builder<ImmutableList<RexLiteral>> newTuples =
@@ -972,7 +1013,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     final LogicalValues newValues =
         LogicalValues.create(values.getCluster(), newRowType,
             newTuples.build());
-    return new TrimResult(newValues, mapping);
+    return result(newValues, mapping);
   }
 
   private Mapping createMapping(ImmutableBitSet fieldsUsed, int fieldCount) {
@@ -1024,7 +1065,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     }
 
     final Mapping mapping = createMapping(fieldsUsed, fieldCount);
-    return new TrimResult(newTableAccessRel, mapping);
+    return result(newTableAccessRel, mapping);
   }
 
   //~ Inner Classes ----------------------------------------------------------
@@ -1043,7 +1084,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
    *
    * <p>For example, consider the mapping for a relational expression that
    * has 4 output columns but only two are being used. The mapping
-   * {2 &rarr; 1, 3 &rarr; 0} would give the following behavior:</p>
+   * {2 &rarr; 1, 3 &rarr; 0} would give the following behavior:
    *
    * <ul>
    * <li>columnsUsed.getSourceCount() returns 4

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/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 31e8f88..bb37de1 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -2847,10 +2847,7 @@ public class SqlToRelConverter {
    * @return Whether to trim unused fields
    */
   public boolean isTrimUnusedFields() {
-    // To work around [CALCITE-842] "Decorrelator gets field offsets confused if
-    // fields have been trimmed", if expansion is disabled, trim fields after
-    // expansion and decorrelation.
-    return trimUnusedFields && expand;
+    return trimUnusedFields;
   }
 
   public void setExpand(boolean expand) {
@@ -3278,14 +3275,8 @@ public class SqlToRelConverter {
     } else {
       qualified = SqlQualified.create(null, 1, null, identifier);
     }
-    RexNode e = bb.lookupExp(qualified);
-    final CorrelationId correlationName;
-    if (e instanceof RexCorrelVariable) {
-      correlationName = ((RexCorrelVariable) e).id;
-    } else {
-      correlationName = null;
-    }
-
+    final RexNode e0 = bb.lookupExp(qualified);
+    RexNode e = e0;
     for (String name : qualified.suffixTranslated()) {
       final boolean caseSensitive = true; // name already fully-qualified
       e = rexBuilder.makeFieldAccess(e, name, caseSensitive);
@@ -3295,10 +3286,11 @@ public class SqlToRelConverter {
       e = adjustInputRef(bb, (RexInputRef) e);
     }
 
-    if (null != correlationName) {
+    if (e0 instanceof RexCorrelVariable) {
       assert e instanceof RexFieldAccess;
       final RexNode prev =
-          bb.mapCorrelateToRex.put(correlationName, (RexFieldAccess) e);
+          bb.mapCorrelateToRex.put(((RexCorrelVariable) e0).id,
+              (RexFieldAccess) e);
       assert prev == null;
     }
     return e;

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index 3615003..51a4a70 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -103,20 +103,20 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.dept d \n"
             + "on e.deptno = d.deptno")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$2], ENAME=[$3], DEPTNO=[$4], DNAME=[$1])\n"
-            + "    JdbcJoin(condition=[=($4, $0)], joinType=[inner])\n"
-            + "      JdbcProject(DEPTNO=[$0], DNAME=[$1])\n"
-            + "        JdbcTableScan(table=[[SCOTT, DEPT]])\n"
+            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], DEPTNO=[$2], DNAME=[$4])\n"
+            + "    JdbcJoin(condition=[=($2, $3)], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], DEPTNO=[$7])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])")
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "      JdbcProject(DEPTNO=[$0], DNAME=[$1])\n"
+            + "        JdbcTableScan(table=[[SCOTT, DEPT]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
-        .planHasSql("SELECT \"t0\".\"EMPNO\", \"t0\".\"ENAME\", "
-            + "\"t0\".\"DEPTNO\", \"t\".\"DNAME\"\n"
-            + "FROM (SELECT \"DEPTNO\", \"DNAME\"\n"
-            + "FROM \"SCOTT\".\"DEPT\") AS \"t\"\n"
-            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"DEPTNO\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" "
+        .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
+            + "\"t\".\"DEPTNO\", \"t0\".\"DNAME\"\n"
+            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"DEPTNO\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
+            + "INNER JOIN (SELECT \"DEPTNO\", \"DNAME\"\n"
+            + "FROM \"SCOTT\".\"DEPT\") AS \"t0\" "
             + "ON \"t\".\"DEPTNO\" = \"t0\".\"DEPTNO\"");
   }
 
@@ -129,20 +129,17 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.salgrade s \n"
             + "on e.sal > s.losal and e.sal < s.hisal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$3], ENAME=[$4], GRADE=[$0])\n"
-            + "    JdbcJoin(condition=[AND(>($5, $1), <($5, $2))], joinType=[inner])\n"
-            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])\n"
+            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], GRADE=[$3])\n"
+            + "    JdbcJoin(condition=[AND(>($2, $4), <($2, $5))], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], SAL=[$5])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])")
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
         .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
             + "\"SALGRADE\".\"GRADE\"\n"
-            + "FROM \"SCOTT\".\"SALGRADE\"\n"
-            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"SAL\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t\" "
-            + "ON \"SALGRADE\".\"LOSAL\" < \"t\".\"SAL\" "
-            + "AND \"SALGRADE\".\"HISAL\" > \"t\".\"SAL\"");
+            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"SAL\"\nFROM \"SCOTT\".\"EMP\") AS \"t\"\n"
+            + "INNER JOIN \"SCOTT\".\"SALGRADE\" ON \"t\".\"SAL\" > \"SALGRADE\".\"LOSAL\" AND \"t\".\"SAL\" < \"SALGRADE\".\"HISAL\"");
   }
 
   @Test public void testNonEquiJoinReverseConditionPlan() {
@@ -151,20 +148,18 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.salgrade s \n"
             + "on s.losal <= e.sal and s.hisal >= e.sal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$3], ENAME=[$4], GRADE=[$0])\n"
-            + "    JdbcJoin(condition=[AND(<=($1, $5), >=($2, $5))], joinType=[inner])\n"
-            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])\n"
+            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], GRADE=[$3])\n"
+            + "    JdbcJoin(condition=[AND(<=($4, $2), >=($5, $2))], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], SAL=[$5])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])")
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
         .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
             + "\"SALGRADE\".\"GRADE\"\n"
-            + "FROM \"SCOTT\".\"SALGRADE\"\n"
-            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"SAL\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t\" "
-            + "ON \"SALGRADE\".\"LOSAL\" <= \"t\".\"SAL\" "
-            + "AND \"SALGRADE\".\"HISAL\" >= \"t\".\"SAL\"");
+            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"SAL\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
+            + "INNER JOIN \"SCOTT\".\"SALGRADE\" ON \"t\".\"SAL\" >= \"SALGRADE\".\"LOSAL\" AND \"t\".\"SAL\" <= \"SALGRADE\".\"HISAL\"");
   }
 
   @Test public void testMixedJoinPlan() {
@@ -173,19 +168,20 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.emp m on  \n"
             + "e.mgr = m.empno and e.sal > m.sal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$2], ENAME=[$3], EMPNO0=[$2], ENAME0=[$3])\n"
-            + "    JdbcJoin(condition=[AND(=($4, $0), >($5, $1))], joinType=[inner])\n"
-            + "      JdbcProject(EMPNO=[$0], SAL=[$5])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], EMPNO0=[$0], ENAME0=[$1])\n"
+            + "    JdbcJoin(condition=[AND(=($2, $4), >($3, $5))], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], MGR=[$3], SAL=[$5])\n"
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "      JdbcProject(EMPNO=[$0], SAL=[$5])\n"
             + "        JdbcTableScan(table=[[SCOTT, EMP]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
-        .planHasSql("SELECT \"t0\".\"EMPNO\", \"t0\".\"ENAME\", "
-            + "\"t0\".\"EMPNO\" AS \"EMPNO0\", \"t0\".\"ENAME\" AS \"ENAME0\"\n"
-            + "FROM (SELECT \"EMPNO\", \"SAL\"\nFROM \"SCOTT\".\"EMP\") AS \"t\"\n"
-            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"MGR\", \"SAL\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" ON \"t\".\"EMPNO\" = \"t0\".\"MGR\" AND \"t\".\"SAL\" < \"t0\".\"SAL\"");
+        .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
+            + "\"t\".\"EMPNO\" AS \"EMPNO0\", \"t\".\"ENAME\" AS \"ENAME0\"\n"
+            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"MGR\", \"SAL\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
+            + "INNER JOIN (SELECT \"EMPNO\", \"SAL\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" ON \"t\".\"MGR\" = \"t0\".\"EMPNO\" AND \"t\".\"SAL\" > \"t0\".\"SAL\"");
   }
 
   @Test public void testMixedJoinWithOrPlan() {
@@ -194,23 +190,20 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.emp m on  \n"
             + "e.mgr = m.empno and (e.sal > m.sal or m.hiredate > e.hiredate)")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$3], ENAME=[$4], EMPNO0=[$3], ENAME0=[$4])\n"
-            + "    JdbcJoin(condition=[AND(=($5, $0), OR(>($7, $2), >($1, $6)))], joinType=[inner])\n"
-            + "      JdbcProject(EMPNO=[$0], HIREDATE=[$4], SAL=[$5])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], EMPNO0=[$0], ENAME0=[$1])\n"
+            + "    JdbcJoin(condition=[AND(=($2, $5), OR(>($4, $7), >($6, $3)))], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], MGR=[$3], HIREDATE=[$4], SAL=[$5])\n"
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "      JdbcProject(EMPNO=[$0], HIREDATE=[$4], SAL=[$5])\n"
             + "        JdbcTableScan(table=[[SCOTT, EMP]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
-        .planHasSql("SELECT \"t0\".\"EMPNO\", \"t0\".\"ENAME\", "
-            + "\"t0\".\"EMPNO\" AS \"EMPNO0\", \"t0\".\"ENAME\" AS \"ENAME0\"\n"
-            + "FROM (SELECT \"EMPNO\", \"HIREDATE\", \"SAL\"\n"
+        .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
+            + "\"t\".\"EMPNO\" AS \"EMPNO0\", \"t\".\"ENAME\" AS \"ENAME0\"\n"
+            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"MGR\", \"HIREDATE\", \"SAL\"\n"
             + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
-            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"MGR\", \"HIREDATE\", \"SAL\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" "
-            + "ON \"t\".\"EMPNO\" = \"t0\".\"MGR\" "
-            + "AND (\"t\".\"SAL\" < \"t0\".\"SAL\" "
-            + "OR \"t\".\"HIREDATE\" > \"t0\".\"HIREDATE\")");
+            + "INNER JOIN (SELECT \"EMPNO\", \"HIREDATE\", \"SAL\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" ON \"t\".\"MGR\" = \"t0\".\"EMPNO\" AND (\"t\".\"SAL\" > \"t0\".\"SAL\" OR \"t\".\"HIREDATE\" < \"t0\".\"HIREDATE\")");
   }
 
   @Test public void testJoin3TablesPlan() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/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 ac9c56b..41a462a 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -3334,10 +3334,12 @@ public class JdbcTest {
       CalciteAssert.hr()
           .query("select count(*) c from \"hr\".\"emps\", \"hr\".\"depts\"")
           .convertContains("LogicalAggregate(group=[{}], C=[COUNT()])\n"
-              + "  LogicalProject($f0=[0])\n"
+              + "  LogicalProject(DUMMY=[0])\n"
               + "    LogicalJoin(condition=[true], joinType=[inner])\n"
-              + "      EnumerableTableScan(table=[[hr, emps]])\n"
-              + "      EnumerableTableScan(table=[[hr, depts]])");
+              + "      LogicalProject(DUMMY=[0])\n"
+              + "        EnumerableTableScan(table=[[hr, emps]])\n"
+              + "      LogicalProject(DUMMY=[0])\n"
+              + "        EnumerableTableScan(table=[[hr, depts]])");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/core/src/test/java/org/apache/calcite/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
index 30ff1e1..be87b1c 100644
--- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
@@ -233,7 +233,7 @@ public class LatticeTest {
           .convertMatches(
               CalciteAssert.checkRel(""
                   + "LogicalAggregate(group=[{}], EXPR$0=[COUNT()])\n"
-                  + "  LogicalProject($f0=[0])\n"
+                  + "  LogicalProject(DUMMY=[0])\n"
                   + "    StarTableScan(table=[[adhoc, star]])\n",
                   counter));
     } catch (RuntimeException e) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index 9cb4240..7e818af 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -518,7 +518,7 @@ public abstract class SqlToRelTestBase {
       }
       if (enableTrim) {
         converter.setTrimUnusedFields(true);
-        root = root.withRel(converter.trimUnusedFields(false, root.rel));
+        root = root.withRel(converter.trimUnusedFields(true, root.rel));
       }
       return root;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
index 101a6ae..f1a7c67 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
@@ -52,9 +52,10 @@ public class EnumerableCorrelateTest {
         .query(
             "select empid, name from emps e where exists (select 1 from depts d where d.deptno=e.deptno)")
         .explainContains(""
-            + "EnumerableCalc(expr#0..5=[{inputs}], empid=[$t0], name=[$t2])\n"
+            + "EnumerableCalc(expr#0..3=[{inputs}], empid=[$t0], name=[$t2])\n"
             + "  EnumerableCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{1}])\n"
-            + "    EnumerableTableScan(table=[[s, emps]])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], proj#0..2=[{exprs}])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
             + "    EnumerableAggregate(group=[{0}])\n"
             + "      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], expr#5=[$cor0], expr#6=[$t5.deptno], expr#7=[=($t0, $t6)], i=[$t4], $condition=[$t7])\n"
             + "        EnumerableTableScan(table=[[s, depts]])")

http://git-wip-us.apache.org/repos/asf/calcite/blob/4b519b98/core/src/test/resources/sql/subquery.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/subquery.iq b/core/src/test/resources/sql/subquery.iq
index b9964b6..cfeb6e5 100644
--- a/core/src/test/resources/sql/subquery.iq
+++ b/core/src/test/resources/sql/subquery.iq
@@ -48,8 +48,8 @@ EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[f
             EnumerableValues(tuples=[[{ 0 }]])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
             EnumerableValues(tuples=[[{ 0 }]])
-    EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
-      EnumerableAggregate(group=[{0}])
+    EnumerableAggregate(group=[{0, 1}])
+      EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
         EnumerableUnion(all=[true])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
             EnumerableValues(tuples=[[{ 0 }]])
@@ -274,6 +274,7 @@ GROUP BY emp.deptno;
 
 !ok
 
+!if (fixed.calcite1045) {
 # Correlated IN sub-query in WHERE clause of JOIN
 select empno from "scott".emp as e
 join "scott".dept as d using (deptno)
@@ -306,6 +307,7 @@ EnumerableCalc(expr#0..5=[{inputs}], EMPNO=[$t0])
     EnumerableCalc(expr#0..2=[{inputs}], DEPTNO=[$t0])
       EnumerableTableScan(table=[[scott, DEPT]])
 !plan
+!}
 
 !if (fixed.calcite1045) {
 # Correlated NOT IN sub-query in WHERE clause of JOIN


[35/50] [abbrv] calcite git commit: [CALCITE-1020] Add MILLISECOND in TimeUnit (Pengcheng Xiong)

Posted by jh...@apache.org.
[CALCITE-1020] Add MILLISECOND in TimeUnit (Pengcheng Xiong)


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

Branch: refs/heads/branch-release
Commit: ebcba3b5ca98d8f894000494a3858ba5a4a269a9
Parents: cabdcf4
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Sat Jan 9 23:05:17 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:52:18 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/calcite/avatica/util/TimeUnit.java   | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/ebcba3b5/avatica/src/main/java/org/apache/calcite/avatica/util/TimeUnit.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/TimeUnit.java b/avatica/src/main/java/org/apache/calcite/avatica/util/TimeUnit.java
index c9a6d42..b249232 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/util/TimeUnit.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/TimeUnit.java
@@ -27,7 +27,12 @@ public enum TimeUnit {
   DAY(false, '-', DateTimeUtils.MILLIS_PER_DAY, null),
   HOUR(false, ' ', DateTimeUtils.MILLIS_PER_HOUR, BigDecimal.valueOf(24)),
   MINUTE(false, ':', DateTimeUtils.MILLIS_PER_MINUTE, BigDecimal.valueOf(60)),
-  SECOND(false, ':', DateTimeUtils.MILLIS_PER_SECOND, BigDecimal.valueOf(60));
+  SECOND(false, ':', DateTimeUtils.MILLIS_PER_SECOND, BigDecimal.valueOf(60)),
+
+  /** Unlike the other units, MILLISECOND may not be the unit of a SQL interval.
+   * Still, it is convenient to use it internally, when converting to and from
+   * UNIX timestamps. */
+  MILLISECOND(false, '.', 1, BigDecimal.valueOf(1));
 
   public final boolean yearMonth;
   public final char separator;


[25/50] [abbrv] calcite git commit: [CALCITE-816] Represent sub-query as a RexNode

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
index c65499a..aacd72a 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
@@ -23,11 +23,13 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
@@ -73,7 +75,7 @@ public final class LogicalJoin extends Join {
    * @param right            Right input
    * @param condition        Join condition
    * @param joinType         Join type
-   * @param variablesStopped Set of names of variables which are set by the
+   * @param variablesSet     Set of variables that are set by the
    *                         LHS and used by the RHS and are not available to
    *                         nodes above this LogicalJoin in the tree
    * @param semiJoinDone     Whether this join has been translated to a
@@ -89,22 +91,30 @@ public final class LogicalJoin extends Join {
       RelNode left,
       RelNode right,
       RexNode condition,
+      Set<CorrelationId> variablesSet,
       JoinRelType joinType,
-      Set<String> variablesStopped,
       boolean semiJoinDone,
       ImmutableList<RelDataTypeField> systemFieldList) {
-    super(cluster, traitSet, left, right, condition, joinType,
-        variablesStopped);
-    assert systemFieldList != null;
+    super(cluster, traitSet, left, right, condition, variablesSet, joinType);
     this.semiJoinDone = semiJoinDone;
-    this.systemFieldList = systemFieldList;
+    this.systemFieldList = Preconditions.checkNotNull(systemFieldList);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public LogicalJoin(RelOptCluster cluster, RelTraitSet traitSet, RelNode left,
+      RelNode right, RexNode condition, JoinRelType joinType,
+      Set<String> variablesStopped, boolean semiJoinDone,
+      ImmutableList<RelDataTypeField> systemFieldList) {
+    this(cluster, traitSet, left, right, condition,
+        CorrelationId.setOf(variablesStopped), joinType, semiJoinDone,
+        systemFieldList);
   }
 
   @Deprecated // to be removed before 2.0
   public LogicalJoin(RelOptCluster cluster, RelNode left, RelNode right,
       RexNode condition, JoinRelType joinType, Set<String> variablesStopped) {
     this(cluster, cluster.traitSetOf(Convention.NONE), left, right, condition,
-        joinType, variablesStopped, false,
+        CorrelationId.setOf(variablesStopped), joinType, false,
         ImmutableList.<RelDataTypeField>of());
   }
 
@@ -113,53 +123,62 @@ public final class LogicalJoin extends Join {
       RexNode condition, JoinRelType joinType, Set<String> variablesStopped,
       boolean semiJoinDone, ImmutableList<RelDataTypeField> systemFieldList) {
     this(cluster, cluster.traitSetOf(Convention.NONE), left, right, condition,
-        joinType, variablesStopped, semiJoinDone, systemFieldList);
+        CorrelationId.setOf(variablesStopped), joinType, semiJoinDone,
+        systemFieldList);
   }
 
   /**
    * Creates a LogicalJoin by parsing serialized output.
    */
   public LogicalJoin(RelInput input) {
-    this(
-        input.getCluster(), input.getInputs().get(0),
-        input.getInputs().get(1), input.getExpression("condition"),
-        input.getEnum("joinType", JoinRelType.class),
-        ImmutableSet.<String>of(), false,
+    this(input.getCluster(), input.getCluster().traitSetOf(Convention.NONE),
+        input.getInputs().get(0), input.getInputs().get(1),
+        input.getExpression("condition"), ImmutableSet.<CorrelationId>of(),
+        input.getEnum("joinType", JoinRelType.class), false,
         ImmutableList.<RelDataTypeField>of());
   }
 
   /** Creates a LogicalJoin, flagged with whether it has been translated to a
    * semi-join. */
   public static LogicalJoin create(RelNode left, RelNode right,
-      RexNode condition, JoinRelType joinType, Set<String> variablesStopped,
+      RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType,
       boolean semiJoinDone, ImmutableList<RelDataTypeField> systemFieldList) {
     final RelOptCluster cluster = left.getCluster();
     final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE);
-    return new LogicalJoin(cluster, traitSet, left, right, condition, joinType,
-        variablesStopped, semiJoinDone, systemFieldList);
+    return new LogicalJoin(cluster, traitSet, left, right, condition,
+        variablesSet, joinType, semiJoinDone, systemFieldList);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public static LogicalJoin create(RelNode left, RelNode right,
+      RexNode condition, JoinRelType joinType, Set<String> variablesStopped,
+      boolean semiJoinDone, ImmutableList<RelDataTypeField> systemFieldList) {
+    return create(left, right, condition, CorrelationId.setOf(variablesStopped),
+        joinType, semiJoinDone, systemFieldList);
   }
 
   /** Creates a LogicalJoin. */
   public static LogicalJoin create(RelNode left, RelNode right,
-      RexNode condition, JoinRelType joinType, Set<String> variablesStopped) {
-    return create(left, right, condition, joinType, variablesStopped, false,
+      RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+    return create(left, right, condition, variablesSet, joinType, false,
         ImmutableList.<RelDataTypeField>of());
   }
 
+  @Deprecated // to be removed before 2.0
+  public static LogicalJoin create(RelNode left, RelNode right,
+      RexNode condition, JoinRelType joinType, Set<String> variablesStopped) {
+    return create(left, right, condition, CorrelationId.setOf(variablesStopped),
+        joinType, false, ImmutableList.<RelDataTypeField>of());
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   @Override public LogicalJoin copy(RelTraitSet traitSet, RexNode conditionExpr,
       RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new LogicalJoin(
-        getCluster(),
-        left,
-        right,
-        conditionExpr,
-        joinType,
-        this.variablesStopped,
-        semiJoinDone,
-        this.systemFieldList);
+    return new LogicalJoin(getCluster(),
+        getCluster().traitSetOf(Convention.NONE), left, right, conditionExpr,
+        variablesSet, joinType, semiJoinDone, systemFieldList);
   }
 
   @Override public RelNode accept(RelShuttle shuttle) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
index db6788a..d77ca13 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
@@ -28,7 +28,6 @@ import org.apache.calcite.rel.core.SetOp;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
 
@@ -76,11 +75,11 @@ public class RelMdUniqueKeys {
     //
     // Further more, the unique bitset coming from the child needs
     // to be mapped to match the output of the project.
-    Map<Integer, Integer> mapInToOutPos = new HashMap<Integer, Integer>();
+    Map<Integer, Integer> mapInToOutPos = new HashMap<>();
 
     List<RexNode> projExprs = rel.getProjects();
 
-    Set<ImmutableBitSet> projUniqueKeySet = new HashSet<ImmutableBitSet>();
+    Set<ImmutableBitSet> projUniqueKeySet = new HashSet<>();
 
     // Build an input to output position map.
     for (int i = 0; i < projExprs.size(); i++) {
@@ -105,7 +104,7 @@ public class RelMdUniqueKeys {
       for (ImmutableBitSet colMask : childUniqueKeySet) {
         ImmutableBitSet.Builder tmpMask = ImmutableBitSet.builder();
         boolean completeKeyProjected = true;
-        for (int bit : BitSets.toIter(colMask)) {
+        for (int bit : colMask) {
           if (mapInToOutPos.containsKey(bit)) {
             tmpMask.set(mapInToOutPos.get(bit));
           } else {
@@ -137,7 +136,7 @@ public class RelMdUniqueKeys {
     // that is undesirable, use RelMetadataQuery.areColumnsUnique() as
     // an alternative way of getting unique key information.
 
-    Set<ImmutableBitSet> retSet = new HashSet<ImmutableBitSet>();
+    Set<ImmutableBitSet> retSet = new HashSet<>();
     Set<ImmutableBitSet> leftSet =
         RelMetadataQuery.getUniqueKeys(left, ignoreNulls);
     Set<ImmutableBitSet> rightSet = null;
@@ -147,7 +146,7 @@ public class RelMdUniqueKeys {
     int nFieldsOnLeft = left.getRowType().getFieldCount();
 
     if (tmpRightSet != null) {
-      rightSet = new HashSet<ImmutableBitSet>();
+      rightSet = new HashSet<>();
       for (ImmutableBitSet colMask : tmpRightSet) {
         ImmutableBitSet.Builder tmpMask = ImmutableBitSet.builder();
         for (int bit : colMask) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java
index 32f4606..52a0474 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/EquiJoin.java
@@ -19,6 +19,7 @@ package org.apache.calcite.rel.rules;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.ImmutableIntList;
@@ -38,7 +39,7 @@ public abstract class EquiJoin extends org.apache.calcite.rel.core.EquiJoin {
       ImmutableIntList rightKeys, JoinRelType joinType,
       Set<String> variablesStopped) {
     super(cluster, traits, left, right, condition, leftKeys, rightKeys,
-        joinType, variablesStopped);
+        CorrelationId.setOf(variablesStopped), joinType);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelateRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelateRule.java
index 5280524..5afbe2a 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinToCorrelateRule.java
@@ -102,10 +102,9 @@ public class JoinToCorrelateRule extends RelOptRule {
     final RelOptCluster cluster = join.getCluster();
     final RexBuilder rexBuilder = cluster.getRexBuilder();
     final RelBuilder relBuilder = call.builder();
-    final int dynInId = cluster.createCorrel();
-    final CorrelationId correlationId = new CorrelationId(dynInId);
+    final CorrelationId correlationId = cluster.createCorrel();
     final RexNode corrVar =
-        rexBuilder.makeCorrel(left.getRowType(), correlationId.getName());
+        rexBuilder.makeCorrel(left.getRowType(), correlationId);
     final ImmutableBitSet.Builder requiredColumns = ImmutableBitSet.builder();
 
     // Replace all references of left input with FieldAccess(corrVar, field)

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java
index c1a7f84..4e522c8 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinUnionTransposeRule.java
@@ -70,7 +70,7 @@ public class JoinUnionTransposeRule extends RelOptRule {
     if (!unionRel.all) {
       return;
     }
-    if (!join.getVariablesStopped().isEmpty()) {
+    if (!join.getVariablesSet().isEmpty()) {
       return;
     }
     // The UNION ALL cannot be on the null generating side

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
new file mode 100644
index 0000000..c3daf49
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
@@ -0,0 +1,365 @@
+/*
+ * 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.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.LogicVisitor;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql2rel.RelDecorrelator;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Pair;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Transform that converts IN, EXISTS and scalar sub-queries into joins.
+ *
+ * <p>Sub-queries are represented by {@link RexSubQuery} expressions.
+ *
+ * <p>A sub-query may or may not be correlated. If a sub-query is correlated,
+ * the wrapped {@link RelNode} will contain a {@link RexCorrelVariable} before
+ * the rewrite, and the product of the rewrite will be a {@link Correlate}.
+ * The Correlate can be removed using {@link RelDecorrelator}.
+ */
+public abstract class SubQueryRemoveRule extends RelOptRule {
+  public static final SubQueryRemoveRule PROJECT =
+      new SubQueryRemoveRule(
+          operand(Project.class, null, RexUtil.SubQueryFinder.PROJECT_PREDICATE,
+              any()),
+          RelFactories.LOGICAL_BUILDER, "SubQueryRemoveRule:Project") {
+        public void onMatch(RelOptRuleCall call) {
+          final Project project = call.rel(0);
+          final RelBuilder builder = call.builder();
+          final RexSubQuery e =
+              RexUtil.SubQueryFinder.find(project.getProjects());
+          assert e != null;
+          final RelOptUtil.Logic logic =
+              LogicVisitor.find(RelOptUtil.Logic.TRUE_FALSE_UNKNOWN,
+                  project.getProjects(), e);
+          builder.push(project.getInput());
+          final int fieldCount = builder.peek().getRowType().getFieldCount();
+          final RexNode target = apply(e, ImmutableSet.<CorrelationId>of(),
+              logic, builder, 1, fieldCount);
+          final RexShuttle shuttle = new ReplaceSubQueryShuttle(e, target);
+          builder.project(shuttle.apply(project.getProjects()),
+              project.getRowType().getFieldNames());
+          call.transformTo(builder.build());
+        }
+      };
+
+  public static final SubQueryRemoveRule FILTER =
+      new SubQueryRemoveRule(
+          operand(Filter.class, null, RexUtil.SubQueryFinder.FILTER_PREDICATE,
+              any()),
+          RelFactories.LOGICAL_BUILDER, "SubQueryRemoveRule:Filter") {
+        public void onMatch(RelOptRuleCall call) {
+          final Filter filter = call.rel(0);
+          final RelBuilder builder = call.builder();
+          final RexSubQuery e =
+              RexUtil.SubQueryFinder.find(filter.getCondition());
+          assert e != null;
+          final RelOptUtil.Logic logic =
+              LogicVisitor.find(RelOptUtil.Logic.TRUE,
+                  ImmutableList.of(filter.getCondition()), e);
+          builder.push(filter.getInput());
+          final int fieldCount = builder.peek().getRowType().getFieldCount();
+          final RexNode target = apply(e, filter.getVariablesSet(), logic,
+              builder, 1, fieldCount);
+          final RexShuttle shuttle = new ReplaceSubQueryShuttle(e, target);
+          builder.filter(shuttle.apply(filter.getCondition()));
+          builder.project(fields(builder, filter.getRowType().getFieldCount()));
+          call.transformTo(builder.build());
+        }
+      };
+
+  public static final SubQueryRemoveRule JOIN =
+      new SubQueryRemoveRule(
+          operand(Join.class, null, RexUtil.SubQueryFinder.JOIN_PREDICATE,
+              any()), RelFactories.LOGICAL_BUILDER, "SubQueryRemoveRule:Join") {
+        public void onMatch(RelOptRuleCall call) {
+          final Join join = call.rel(0);
+          final RelBuilder builder = call.builder();
+          final RexSubQuery e =
+              RexUtil.SubQueryFinder.find(join.getCondition());
+          assert e != null;
+          final RelOptUtil.Logic logic =
+              LogicVisitor.find(RelOptUtil.Logic.TRUE,
+                  ImmutableList.of(join.getCondition()), e);
+          builder.push(join.getLeft());
+          builder.push(join.getRight());
+          final int fieldCount = join.getRowType().getFieldCount();
+          final RexNode target = apply(e, ImmutableSet.<CorrelationId>of(),
+              logic, builder, 2, fieldCount);
+          final RexShuttle shuttle = new ReplaceSubQueryShuttle(e, target);
+          builder.join(join.getJoinType(), shuttle.apply(join.getCondition()));
+          builder.project(fields(builder, join.getRowType().getFieldCount()));
+          call.transformTo(builder.build());
+        }
+      };
+
+  private SubQueryRemoveRule(RelOptRuleOperand operand,
+      RelBuilderFactory relBuilderFactory,
+      String description) {
+    super(operand, relBuilderFactory, description);
+  }
+
+  protected RexNode apply(RexSubQuery e, Set<CorrelationId> variablesSet,
+      RelOptUtil.Logic logic,
+      RelBuilder builder, int inputCount, int offset) {
+    switch (e.getKind()) {
+    case SCALAR_QUERY:
+      builder.push(e.rel);
+      final Boolean unique = RelMetadataQuery.areColumnsUnique(builder.peek(),
+          ImmutableBitSet.of());
+      if (unique == null || !unique) {
+        builder.aggregate(builder.groupKey(),
+            builder.aggregateCall(SqlStdOperatorTable.SINGLE_VALUE, false, null,
+                null, builder.field(0)));
+      }
+      builder.join(JoinRelType.LEFT);
+      return field(builder, inputCount, offset);
+
+    case IN:
+    case EXISTS:
+      // Most general case, where the left and right keys might have nulls, and
+      // caller requires 3-valued logic return.
+      //
+      // select e.deptno, e.deptno in (select deptno from emp)
+      //
+      // becomes
+      //
+      // select e.deptno,
+      //   case
+      //   when ct.c = 0 then false
+      //   when dt.i is not null then true
+      //   when e.deptno is null then null
+      //   when ct.ck < ct.c then null
+      //   else false
+      //   end
+      // from e
+      // left join (
+      //   (select count(*) as c, count(deptno) as ck from emp) as ct
+      //   cross join (select distinct deptno, true as i from emp)) as dt
+      //   on e.deptno = dt.deptno
+      //
+      // If keys are not null we can remove "ct" and simplify to
+      //
+      // select e.deptno,
+      //   case
+      //   when dt.i is not null then true
+      //   else false
+      //   end
+      // from e
+      // left join (select distinct deptno, true as i from emp) as dt
+      //   on e.deptno = dt.deptno
+      //
+      // We could further simplify to
+      //
+      // select e.deptno,
+      //   dt.i is not null
+      // from e
+      // left join (select distinct deptno, true as i from emp) as dt
+      //   on e.deptno = dt.deptno
+      //
+      // but have not yet.
+      //
+      // If the logic is TRUE we can just kill the record if the condition
+      // evaluates to FALSE or UNKNOWN. Thus the query simplifies to an inner
+      // join:
+      //
+      // select e.deptno,
+      //   true
+      // from e
+      // inner join (select distinct deptno from emp) as dt
+      //   on e.deptno = dt.deptno
+      //
+
+      builder.push(e.rel);
+      final List<RexNode> fields = new ArrayList<>();
+      switch (e.getKind()) {
+      case IN:
+        fields.addAll(builder.fields());
+      }
+
+      // First, the cross join
+      switch (logic) {
+      case TRUE_FALSE_UNKNOWN:
+      case UNKNOWN_AS_TRUE:
+        if (!variablesSet.isEmpty()) {
+          // We have not yet figured out how to include "ct" in a query if
+          // the source relation "e.rel" is correlated. So, dodge the issue:
+          // we pretend that the join key is NOT NULL.
+          //
+          // We will get wrong results in correlated IN where the join
+          // key has nulls. E.g.
+          //
+          //   SELECT *
+          //   FROM emp
+          //   WHERE mgr NOT IN (
+          //     SELECT mgr
+          //     FROM emp AS e2
+          //     WHERE
+          logic = RelOptUtil.Logic.TRUE_FALSE;
+          break;
+        }
+        builder.aggregate(builder.groupKey(),
+            builder.count(false, "c"),
+            builder.aggregateCall(SqlStdOperatorTable.COUNT, false, null, "ck",
+                builder.fields()));
+        builder.as("ct");
+        builder.join(JoinRelType.INNER);
+        offset += 2;
+        builder.push(e.rel);
+        break;
+      }
+
+      // Now the left join
+      switch (logic) {
+      case TRUE:
+        if (fields.isEmpty()) {
+          builder.project(builder.alias(builder.literal(true), "i"));
+          builder.aggregate(builder.groupKey(0));
+        } else {
+          builder.aggregate(builder.groupKey(fields));
+        }
+        break;
+      default:
+        fields.add(builder.alias(builder.literal(true), "i"));
+        builder.project(fields);
+        builder.distinct();
+      }
+      builder.as("dt");
+      final List<RexNode> conditions = new ArrayList<>();
+      for (Pair<RexNode, RexNode> pair
+          : Pair.zip(e.getOperands(), builder.fields())) {
+        conditions.add(
+            builder.equals(pair.left, RexUtil.shift(pair.right, offset)));
+      }
+      switch (logic) {
+      case TRUE:
+        builder.join(JoinRelType.INNER, builder.and(conditions), variablesSet);
+        return builder.literal(true);
+      }
+      builder.join(JoinRelType.LEFT, builder.and(conditions), variablesSet);
+
+      final List<RexNode> keyIsNulls = new ArrayList<>();
+      for (RexNode operand : e.getOperands()) {
+        if (operand.getType().isNullable()) {
+          keyIsNulls.add(builder.isNull(operand));
+        }
+      }
+      final ImmutableList.Builder<RexNode> operands = ImmutableList.builder();
+      switch (logic) {
+      case TRUE_FALSE_UNKNOWN:
+      case UNKNOWN_AS_TRUE:
+        operands.add(
+            builder.equals(builder.field("ct", "c"), builder.literal(0)),
+            builder.literal(false));
+        break;
+      }
+      operands.add(builder.isNotNull(builder.field("dt", "i")),
+          builder.literal(true));
+      if (!keyIsNulls.isEmpty()) {
+        operands.add(builder.or(keyIsNulls), builder.literal(null));
+      }
+      Boolean b = true;
+      switch (logic) {
+      case TRUE_FALSE_UNKNOWN:
+        b = null;
+        // fall through
+      case UNKNOWN_AS_TRUE:
+        operands.add(
+            builder.call(SqlStdOperatorTable.LESS_THAN,
+                builder.field("ct", "ck"), builder.field("ct", "c")),
+            builder.literal(b));
+        break;
+      }
+      operands.add(builder.literal(false));
+      return builder.call(SqlStdOperatorTable.CASE, operands.build());
+
+    default:
+      throw new AssertionError(e.getKind());
+    }
+  }
+
+  /** Returns a reference to a particular field, by offset, across several
+   * inputs on a {@link RelBuilder}'s stack. */
+  private RexInputRef field(RelBuilder builder, int inputCount, int offset) {
+    for (int inputOrdinal = 0;;) {
+      final RelNode r = builder.peek(inputCount, inputOrdinal);
+      if (offset < r.getRowType().getFieldCount()) {
+        return builder.field(inputCount, inputOrdinal, offset);
+      }
+      ++inputOrdinal;
+      offset -= r.getRowType().getFieldCount();
+    }
+  }
+
+  /** Returns a list of expressions that project the first {@code fieldCount}
+   * fields of the top input on a {@link RelBuilder}'s stack. */
+  private static List<RexNode> fields(RelBuilder builder, int fieldCount) {
+    final List<RexNode> projects = new ArrayList<>();
+    for (int i = 0; i < fieldCount; i++) {
+      projects.add(builder.field(i));
+    }
+    return projects;
+  }
+
+  /** Shuttle that replaces occurrences of a given
+   * {@link org.apache.calcite.rex.RexSubQuery} with a replacement
+   * expression. */
+  private static class ReplaceSubQueryShuttle extends RexShuttle {
+    private final RexSubQuery subQuery;
+    private final RexNode replacement;
+
+    public ReplaceSubQueryShuttle(RexSubQuery subQuery, RexNode replacement) {
+      this.subQuery = subQuery;
+      this.replacement = replacement;
+    }
+
+    @Override public RexNode visitSubQuery(RexSubQuery subQuery) {
+      return RexUtil.eq(subQuery, this.subQuery) ? replacement : subQuery;
+    }
+  }
+}
+
+// End SubQueryRemoveRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 48e7bbf..acb4eb6 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
@@ -251,13 +251,15 @@ public class StreamRules {
       final RelNode right = join.getRight();
 
       final LogicalDelta rightWithDelta = LogicalDelta.create(right);
-      final LogicalJoin joinL = LogicalJoin.create(left, rightWithDelta, join.getCondition(),
-          join.getJoinType(), join.getVariablesStopped(), join.isSemiJoinDone(),
+      final LogicalJoin joinL = LogicalJoin.create(left, rightWithDelta,
+          join.getCondition(), join.getVariablesSet(), join.getJoinType(),
+          join.isSemiJoinDone(),
           ImmutableList.copyOf(join.getSystemFieldList()));
 
       final LogicalDelta leftWithDelta = LogicalDelta.create(left);
-      final LogicalJoin joinR = LogicalJoin.create(leftWithDelta, right, join.getCondition(),
-          join.getJoinType(), join.getVariablesStopped(), join.isSemiJoinDone(),
+      final LogicalJoin joinR = LogicalJoin.create(leftWithDelta, right,
+          join.getCondition(), join.getVariablesSet(), join.getJoinType(),
+          join.isSemiJoinDone(),
           ImmutableList.copyOf(join.getSystemFieldList()));
 
       List<RelNode> inputsToUnion = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/LogicVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/LogicVisitor.java b/core/src/main/java/org/apache/calcite/rex/LogicVisitor.java
new file mode 100644
index 0000000..610a004
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rex/LogicVisitor.java
@@ -0,0 +1,158 @@
+/*
+ * 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.rex;
+
+import org.apache.calcite.plan.RelOptUtil.Logic;
+
+import com.google.common.collect.Iterables;
+
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Visitor pattern for traversing a tree of {@link RexNode} objects.
+ */
+public class LogicVisitor implements RexBiVisitor<Logic, Logic> {
+  private final RexNode seek;
+  private final Collection<Logic> logicCollection;
+
+  /** Creates a LogicVisitor. */
+  private LogicVisitor(RexNode seek, Collection<Logic> logicCollection) {
+    this.seek = seek;
+    this.logicCollection = logicCollection;
+  }
+
+  /** Finds a suitable logic for evaluating {@code seek} within a list of
+   * expressions.
+   *
+   * <p>Chooses a logic that is safe (that is, gives the right
+   * answer) with the fewest possibilities (that is, we prefer one that
+   * returns [true as true, false as false, unknown as false] over one that
+   * distinguishes false from unknown).
+   */
+  public static Logic find(Logic logic, List<RexNode> nodes,
+      RexNode seek) {
+    final Set<Logic> set = EnumSet.noneOf(Logic.class);
+    final LogicVisitor visitor = new LogicVisitor(seek, set);
+    for (RexNode node : nodes) {
+      node.accept(visitor, logic);
+    }
+    switch (set.size()) {
+    case 0:
+      throw new IllegalArgumentException("not found: " + seek);
+    case 1:
+      return Iterables.getOnlyElement(set);
+    default:
+      return Logic.TRUE_FALSE_UNKNOWN;
+    }
+  }
+
+  public static void collect(RexNode node, RexNode seek, Logic logic,
+      Collection<Logic> logicCollection) {
+    node.accept(new LogicVisitor(seek, logicCollection), logic);
+  }
+
+  public Logic visitCall(RexCall call, Logic logic) {
+    final Logic arg0 = logic;
+    switch (call.getKind()) {
+    case IS_NOT_NULL:
+    case IS_NULL:
+      logic = Logic.TRUE_FALSE_UNKNOWN;
+      break;
+    case IS_TRUE:
+    case IS_NOT_TRUE:
+      logic = Logic.UNKNOWN_AS_FALSE;
+      break;
+    case IS_FALSE:
+    case IS_NOT_FALSE:
+      logic = Logic.UNKNOWN_AS_TRUE;
+      break;
+    case NOT:
+      logic = logic.negate();
+      break;
+    case CASE:
+      logic = Logic.TRUE_FALSE_UNKNOWN;
+      break;
+    }
+    switch (logic) {
+    case TRUE:
+      switch (call.getKind()) {
+      case AND:
+        break;
+      default:
+        logic = Logic.TRUE_FALSE;
+      }
+    }
+    for (RexNode operand : call.operands) {
+      operand.accept(this, logic);
+    }
+    return end(call, arg0);
+  }
+
+  private Logic end(RexNode node, Logic arg) {
+    if (RexUtil.eq(node, seek)) {
+      logicCollection.add(arg);
+    }
+    return arg;
+  }
+
+  public Logic visitInputRef(RexInputRef inputRef, Logic arg) {
+    return end(inputRef, arg);
+  }
+
+  public Logic visitLocalRef(RexLocalRef localRef, Logic arg) {
+    return end(localRef, arg);
+  }
+
+  public Logic visitLiteral(RexLiteral literal, Logic arg) {
+    return end(literal, arg);
+  }
+
+  public Logic visitOver(RexOver over, Logic arg) {
+    return end(over, arg);
+  }
+
+  public Logic visitCorrelVariable(RexCorrelVariable correlVariable,
+      Logic arg) {
+    return end(correlVariable, arg);
+  }
+
+  public Logic visitDynamicParam(RexDynamicParam dynamicParam, Logic arg) {
+    return end(dynamicParam, arg);
+  }
+
+  public Logic visitRangeRef(RexRangeRef rangeRef, Logic arg) {
+    return end(rangeRef, arg);
+  }
+
+  public Logic visitFieldAccess(RexFieldAccess fieldAccess, Logic arg) {
+    return end(fieldAccess, arg);
+  }
+
+  public Logic visitSubQuery(RexSubQuery subQuery, Logic arg) {
+    if (!subQuery.getType().isNullable()) {
+      if (arg == Logic.TRUE_FALSE_UNKNOWN) {
+        arg = Logic.TRUE_FALSE;
+      }
+    }
+    return end(subQuery, arg);
+  }
+}
+
+// End LogicVisitor.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexBiVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBiVisitor.java b/core/src/main/java/org/apache/calcite/rex/RexBiVisitor.java
new file mode 100644
index 0000000..8f7e0d3
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rex/RexBiVisitor.java
@@ -0,0 +1,52 @@
+/*
+ * 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.rex;
+
+/**
+ * Visitor pattern for traversing a tree of {@link RexNode} objects
+ * and passing a payload to each.
+ *
+ * @see RexVisitor
+ *
+ * @param <R> Return type
+ * @param <P> Payload type
+ */
+public interface RexBiVisitor<R, P> {
+  //~ Methods ----------------------------------------------------------------
+
+  R visitInputRef(RexInputRef inputRef, P arg);
+
+  R visitLocalRef(RexLocalRef localRef, P arg);
+
+  R visitLiteral(RexLiteral literal, P arg);
+
+  R visitCall(RexCall call, P arg);
+
+  R visitOver(RexOver over, P arg);
+
+  R visitCorrelVariable(RexCorrelVariable correlVariable, P arg);
+
+  R visitDynamicParam(RexDynamicParam dynamicParam, P arg);
+
+  R visitRangeRef(RexRangeRef rangeRef, P arg);
+
+  R visitFieldAccess(RexFieldAccess fieldAccess, P arg);
+
+  R visitSubQuery(RexSubQuery subQuery, P arg);
+}
+
+// End RexBiVisitor.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index b8c64e3..dd147a5 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -23,6 +23,7 @@ import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -431,14 +432,14 @@ public class RexBuilder {
   /**
    * Creates an expression referencing a correlation variable.
    *
+   * @param id Name of variable
    * @param type Type of variable
-   * @param name Name of variable
    * @return Correlation variable
    */
   public RexNode makeCorrel(
       RelDataType type,
-      String name) {
-    return new RexCorrelVariable(name, type);
+      CorrelationId id) {
+    return new RexCorrelVariable(id, type);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 7cf8255..c843651 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -45,9 +45,9 @@ import java.util.List;
 public class RexCall extends RexNode {
   //~ Instance fields --------------------------------------------------------
 
-  private final SqlOperator op;
+  public final SqlOperator op;
   public final ImmutableList<RexNode> operands;
-  private final RelDataType type;
+  public final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -108,6 +108,10 @@ public class RexCall extends RexNode {
     return visitor.visitCall(this);
   }
 
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitCall(this, arg);
+  }
+
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java b/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
index ea1dade..4880c8e 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCorrelVariable.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.rex;
 
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 
@@ -27,12 +28,15 @@ import org.apache.calcite.sql.SqlKind;
  * assigned a value, and the other side of the join is restarted.</p>
  */
 public class RexCorrelVariable extends RexVariable {
+  public final CorrelationId id;
+
   //~ Constructors -----------------------------------------------------------
 
   RexCorrelVariable(
-      String varName,
+      CorrelationId id,
       RelDataType type) {
-    super(varName, type);
+    super(id.getName(), type);
+    this.id = id;
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -41,6 +45,10 @@ public class RexCorrelVariable extends RexVariable {
     return visitor.visitCorrelVariable(this);
   }
 
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitCorrelVariable(this, arg);
+  }
+
   @Override public SqlKind getKind() {
     return SqlKind.CORREL_VARIABLE;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java b/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java
index cd51cc9..6bbd692 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexDynamicParam.java
@@ -55,6 +55,10 @@ public class RexDynamicParam extends RexVariable {
   public <R> R accept(RexVisitor<R> visitor) {
     return visitor.visitDynamicParam(this);
   }
+
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitDynamicParam(this, arg);
+  }
 }
 
 // End RexDynamicParam.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java b/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java
index 09b9058..658ffc5 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexFieldAccess.java
@@ -81,6 +81,10 @@ public class RexFieldAccess extends RexNode {
     return visitor.visitFieldAccess(this);
   }
 
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitFieldAccess(this, arg);
+  }
+
   /**
    * Returns the expression whose field is being accessed.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexInputRef.java b/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
index 0c475db..8019ad6 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
@@ -112,6 +112,10 @@ public class RexInputRef extends RexSlot {
     return visitor.visitInputRef(this);
   }
 
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitInputRef(this, arg);
+  }
+
   /**
    * Creates a name for an input reference, of the form "$index". If the index
    * is low, uses a cache of common names, to reduce gc.

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
index bd07072..acdebec 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -279,7 +279,7 @@ public class RexLiteral extends RexNode {
       }
       return litmus.succeed();
     } else if (o instanceof Map) {
-      final Map<Object, Object> map = (Map) o;
+      @SuppressWarnings("unchecked") final Map<Object, Object> map = (Map) o;
       for (Map.Entry entry : map.entrySet()) {
         if (!validConstant(entry.getKey(), litmus)) {
           return litmus.fail("not a constant: " + entry.getKey());
@@ -649,15 +649,17 @@ public class RexLiteral extends RexNode {
         && (((RexLiteral) node).value == null);
   }
 
-  private static boolean equals(
-      Object o1,
-      Object o2) {
+  private static boolean equals(Object o1, Object o2) {
     return (o1 == null) ? (o2 == null) : o1.equals(o2);
   }
 
   public <R> R accept(RexVisitor<R> visitor) {
     return visitor.visitLiteral(this);
   }
+
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitLiteral(this, arg);
+  }
 }
 
 // End RexLiteral.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java b/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java
index 9d4d1cc..57a3aff 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLocalRef.java
@@ -77,6 +77,10 @@ public class RexLocalRef extends RexSlot {
     return visitor.visitLocalRef(this);
   }
 
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitLocalRef(this, arg);
+  }
+
   private static String createName(int index) {
     return NAMES.get(index);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexNode.java b/core/src/main/java/org/apache/calcite/rex/RexNode.java
index 0922116..95df460 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexNode.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexNode.java
@@ -90,6 +90,12 @@ public abstract class RexNode {
    * which applies a visitor to several expressions simultaneously.
    */
   public abstract <R> R accept(RexVisitor<R> visitor);
+
+  /**
+   * Accepts a visitor with a payload, dispatching to the right overloaded
+   * {@link RexBiVisitor#visitInputRef(RexInputRef, Object)} visitXxx} method.
+   */
+  public abstract <R, P> R accept(RexBiVisitor<R, P> visitor, P arg);
 }
 
 // End RexNode.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexOver.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexOver.java b/core/src/main/java/org/apache/calcite/rex/RexOver.java
index 3b45f36..5c2c881 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexOver.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexOver.java
@@ -89,6 +89,10 @@ public class RexOver extends RexCall {
     return visitor.visitOver(this);
   }
 
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitOver(this, arg);
+  }
+
   /**
    * Returns whether an expression contains an OVER clause.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexProgram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgram.java b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
index 664c092..30382c5 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgram.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
@@ -859,6 +859,10 @@ public class RexProgram {
       return expr.accept(this);
     }
 
+    @Override public Boolean visitOver(RexOver over) {
+      return false;
+    }
+
     @Override public Boolean visitCorrelVariable(RexCorrelVariable correlVariable) {
       // Correlating variables are constant WITHIN A RESTART, so that's
       // good enough.

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java b/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java
index bf2104c..04aa277 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexRangeRef.java
@@ -70,6 +70,10 @@ public class RexRangeRef extends RexNode {
   public <R> R accept(RexVisitor<R> visitor) {
     return visitor.visitRangeRef(this);
   }
+
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitRangeRef(this, arg);
+  }
 }
 
 // End RexRangeRef.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexShuttle.java b/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
index e3c0302..f0e5820 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
@@ -78,6 +78,16 @@ public class RexShuttle implements RexVisitor<RexNode> {
     }
   }
 
+  public RexNode visitSubQuery(RexSubQuery subQuery) {
+    boolean[] update = {false};
+    List<RexNode> clonedOperands = visitList(subQuery.operands, update);
+    if (update[0]) {
+      return subQuery.clone(subQuery.getType(), clonedOperands);
+    } else {
+      return subQuery;
+    }
+  }
+
   public RexNode visitCall(final RexCall call) {
     boolean[] update = {false};
     List<RexNode> clonedOperands = visitList(call.operands, update);
@@ -238,7 +248,7 @@ public class RexShuttle implements RexVisitor<RexNode> {
     if (exprList == null) {
       return null;
     }
-    final List<T> list2 = new ArrayList<T>(exprList);
+    final List<T> list2 = new ArrayList<>(exprList);
     if (mutate(list2)) {
       return list2;
     } else {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java b/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java
new file mode 100644
index 0000000..d8fff4e
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java
@@ -0,0 +1,115 @@
+/*
+ * 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.rex;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Scalar expression that represents an IN, EXISTS or scalar sub-query.
+ */
+public class RexSubQuery extends RexCall {
+  public final RelNode rel;
+
+  private RexSubQuery(RelDataType type, SqlOperator op,
+      ImmutableList<RexNode> operands, RelNode rel) {
+    super(type, op, operands);
+    this.rel = rel;
+  }
+
+  /** Creates an IN sub-query. */
+  public static RexSubQuery in(RelNode rel, ImmutableList<RexNode> nodes) {
+    assert rel.getRowType().getFieldCount() == nodes.size();
+    final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory();
+    boolean nullable = false;
+    for (RexNode node : nodes) {
+      if (node.getType().isNullable()) {
+        nullable = true;
+      }
+    }
+    for (RelDataTypeField field : rel.getRowType().getFieldList()) {
+      if (field.getType().isNullable()) {
+        nullable = true;
+      }
+    }
+    final RelDataType type =
+        typeFactory.createTypeWithNullability(
+            typeFactory.createSqlType(SqlTypeName.BOOLEAN), nullable);
+    return new RexSubQuery(type, SqlStdOperatorTable.IN, nodes, rel);
+  }
+
+  /** Creates an EXISTS sub-query. */
+  public static RexSubQuery exists(RelNode rel) {
+    final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory();
+    final RelDataType type = typeFactory.createSqlType(SqlTypeName.BOOLEAN);
+    return new RexSubQuery(type, SqlStdOperatorTable.EXISTS,
+        ImmutableList.<RexNode>of(), rel);
+  }
+
+  /** Creates a scalar sub-query. */
+  public static RexSubQuery scalar(RelNode rel) {
+    final List<RelDataTypeField> fieldList = rel.getRowType().getFieldList();
+    assert fieldList.size() == 1;
+    final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory();
+    final RelDataType type =
+        typeFactory.createTypeWithNullability(fieldList.get(0).getType(), true);
+    return new RexSubQuery(type, SqlStdOperatorTable.SCALAR_QUERY,
+        ImmutableList.<RexNode>of(), rel);
+  }
+
+  public <R> R accept(RexVisitor<R> visitor) {
+    return visitor.visitSubQuery(this);
+  }
+
+  public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitSubQuery(this, arg);
+  }
+
+  @Override protected String computeDigest(boolean withType) {
+    StringBuilder sb = new StringBuilder(op.getName());
+    sb.append("(");
+    for (RexNode operand : operands) {
+      sb.append(operand.toString());
+      sb.append(", ");
+    }
+    sb.append("{\n");
+    sb.append(RelOptUtil.toString(rel));
+    sb.append("})");
+    return sb.toString();
+  }
+
+  @Override public RexSubQuery clone(RelDataType type, List<RexNode> operands) {
+    return new RexSubQuery(type, getOperator(),
+        ImmutableList.copyOf(operands), rel);
+  }
+
+  public RexSubQuery clone(RelNode rel) {
+    return new RexSubQuery(type, getOperator(), operands, rel);
+  }
+}
+
+// End RexSubQuery.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index b2de640..b766590 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -22,6 +22,9 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
@@ -260,6 +263,10 @@ public class RexUtil {
       return false;
     }
 
+    public Boolean visitSubQuery(RexSubQuery subQuery) {
+      return false;
+    }
+
     public Boolean visitCorrelVariable(RexCorrelVariable correlVariable) {
       // Correlating variables change when there is an internal restart.
       // Not good enough for our purposes.
@@ -1248,7 +1255,7 @@ public class RexUtil {
     Iterator<RexNode> iterator = targets.iterator();
     while (iterator.hasNext()) {
       RexNode next = iterator.next();
-      if (equivalent(next, e)) {
+      if (eq(next, e)) {
         ++count;
         iterator.remove();
       }
@@ -1256,11 +1263,12 @@ public class RexUtil {
     return count > 0;
   }
 
-  /** Returns whether two expressions are equivalent. */
-  private static boolean equivalent(RexNode e1, RexNode e2) {
-    // TODO: make broader;
-    // 1. 'x = y' should be equivalent to 'y = x'.
-    // 2. 'c2 and c1' should be equivalent to 'c1 and c2'.
+  /** Returns whether two {@link RexNode}s are structurally equal.
+   *
+   * <p>This method considers structure, not semantics. 'x &lt; y' is not
+   * equivalent to 'y &gt; x'.
+   */
+  public static boolean eq(RexNode e1, RexNode e2) {
     return e1 == e2 || e1.toString().equals(e2.toString());
   }
 
@@ -1341,8 +1349,12 @@ public class RexUtil {
     }
     switch (a.getKind()) {
     case NOT:
-      // NOT x IS TRUE ==> x IS NOT TRUE
+      // (NOT x) IS TRUE ==> x IS FALSE
       // Similarly for IS NOT TRUE, IS FALSE, etc.
+      //
+      // Note that
+      //   (NOT x) IS TRUE !=> x IS FALSE
+      // because of null values.
       return simplify(rexBuilder,
           rexBuilder.makeCall(op(kind.negate()),
               ((RexCall) a).getOperands().get(0)));
@@ -1990,7 +2002,7 @@ public class RexUtil {
   }
 
   /** Visitor that throws {@link org.apache.calcite.util.Util.FoundOne} if
-   * there an expression contains a {@link RexCorrelVariable}. */
+   * applied to an expression that contains a {@link RexCorrelVariable}. */
   private static class CorrelationFinder extends RexVisitorImpl<Void> {
     static final CorrelationFinder INSTANCE = new CorrelationFinder();
 
@@ -2030,6 +2042,81 @@ public class RexUtil {
       throw new AssertionError("mismatched type " + ref + " " + rightType);
     }
   }
+
+  /** Visitor that throws {@link org.apache.calcite.util.Util.FoundOne} if
+   * applied to an expression that contains a {@link RexSubQuery}. */
+  public static class SubQueryFinder extends RexVisitorImpl<Void> {
+    public static final SubQueryFinder INSTANCE = new SubQueryFinder();
+
+    /** Returns whether a {@link Project} contains a sub-query. */
+    public static final Predicate<Project> PROJECT_PREDICATE =
+        new Predicate<Project>() {
+          public boolean apply(Project project) {
+            for (RexNode node : project.getProjects()) {
+              try {
+                node.accept(INSTANCE);
+              } catch (Util.FoundOne e) {
+                return true;
+              }
+            }
+            return false;
+          }
+        };
+
+    /** Returns whether a {@link Filter} contains a sub-query. */
+    public static final Predicate<Filter> FILTER_PREDICATE =
+        new Predicate<Filter>() {
+          public boolean apply(Filter filter) {
+            try {
+              filter.getCondition().accept(INSTANCE);
+              return false;
+            } catch (Util.FoundOne e) {
+              return true;
+            }
+          }
+        };
+
+    /** Returns whether a {@link Join} contains a sub-query. */
+    public static final Predicate<Join> JOIN_PREDICATE =
+        new Predicate<Join>() {
+          public boolean apply(Join join) {
+            try {
+              join.getCondition().accept(INSTANCE);
+              return false;
+            } catch (Util.FoundOne e) {
+              return true;
+            }
+          }
+        };
+
+    private SubQueryFinder() {
+      super(true);
+    }
+
+    @Override public Void visitSubQuery(RexSubQuery subQuery) {
+      throw new Util.FoundOne(subQuery);
+    }
+
+    public static RexSubQuery find(Iterable<RexNode> nodes) {
+      for (RexNode node : nodes) {
+        try {
+          node.accept(INSTANCE);
+        } catch (Util.FoundOne e) {
+          return (RexSubQuery) e.getNode();
+        }
+      }
+      return null;
+    }
+
+    public static RexSubQuery find(RexNode node) {
+      try {
+        node.accept(INSTANCE);
+        return null;
+      } catch (Util.FoundOne e) {
+        return (RexSubQuery) e.getNode();
+      }
+    }
+  }
 }
 
 // End RexUtil.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexVisitor.java b/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
index fd3d6b6..4c95647 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
@@ -45,6 +45,8 @@ public interface RexVisitor<R> {
   R visitRangeRef(RexRangeRef rangeRef);
 
   R visitFieldAccess(RexFieldAccess fieldAccess);
+
+  R visitSubQuery(RexSubQuery subQuery);
 }
 
 // End RexVisitor.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java b/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
index fcc6ff7..ec31f26 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
@@ -98,6 +98,18 @@ public class RexVisitorImpl<R> implements RexVisitor<R> {
     return expr.accept(this);
   }
 
+  public R visitSubQuery(RexSubQuery subQuery) {
+    if (!deep) {
+      return null;
+    }
+
+    R r = null;
+    for (RexNode operand : subQuery.operands) {
+      r = operand.accept(this);
+    }
+    return r;
+  }
+
   /**
    * <p>Visits an array of expressions, returning the logical 'and' of their
    * results.

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
index 8d5f380..ee5c02c 100644
--- a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
+++ b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
@@ -26,7 +26,7 @@ import com.google.common.collect.ImmutableList;
  * <p>Given a user-defined schema that implements the {@link Schema} interface,
  * Calcite creates a wrapper that implements the {@code SchemaPlus} interface.
  * This provides extra functionality, such as access to tables that have been
- * added explicitly.</p>
+ * added explicitly.
  *
  * <p>A user-defined schema does not need to implement this interface, but by
  * the time a schema is passed to a method in a user-defined schema or

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 37a757d..270a311 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -913,15 +913,15 @@ public enum SqlKind {
   public SqlKind negate() {
     switch (this) {
     case IS_TRUE:
-      return IS_NOT_TRUE;
+      return IS_FALSE;
     case IS_FALSE:
-      return IS_NOT_FALSE;
+      return IS_TRUE;
     case IS_NULL:
       return IS_NOT_NULL;
     case IS_NOT_TRUE:
-      return IS_TRUE;
+      return IS_NOT_FALSE;
     case IS_NOT_FALSE:
-      return IS_FALSE;
+      return IS_NOT_TRUE;
     case IS_NOT_NULL:
       return IS_NULL;
     case IS_DISTINCT_FROM:

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 8b92b91..00b8611 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
@@ -1927,6 +1927,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       if (newRight != right) {
         join.setRight(newRight);
       }
+      registerSubqueries(joinScope, join.getCondition());
       final JoinNamespace joinNamespace = new JoinNamespace(this, join);
       registerNamespace(null, null, joinNamespace, forceNullable);
       return join;

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/sql2rel/DeduplicateCorrelateVariables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/DeduplicateCorrelateVariables.java b/core/src/main/java/org/apache/calcite/sql2rel/DeduplicateCorrelateVariables.java
index a24924c..43661ef 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/DeduplicateCorrelateVariables.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/DeduplicateCorrelateVariables.java
@@ -18,12 +18,14 @@ package org.apache.calcite.sql2rel;
 
 import org.apache.calcite.rel.RelHomogeneousShuttle;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCorrelVariable;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
 
-import java.util.Set;
+import com.google.common.collect.ImmutableSet;
 
 /**
  * Rewrites relations to ensure the same correlation is referenced by the same
@@ -32,39 +34,64 @@ import java.util.Set;
 public class DeduplicateCorrelateVariables extends RelHomogeneousShuttle {
   private final RexShuttle dedupRex;
 
+  /** Creates a DeduplicateCorrelateVariables. */
+  private DeduplicateCorrelateVariables(RexBuilder builder,
+      CorrelationId canonicalId, ImmutableSet<CorrelationId> alternateIds) {
+    dedupRex = new DeduplicateCorrelateVariablesShuttle(builder,
+        canonicalId, alternateIds, this);
+  }
+
+  /**
+   * Rewrites a relational expression, replacing alternate correlation variables
+   * with a canonical correlation variable.
+   */
+  public static RelNode go(RexBuilder builder, CorrelationId canonicalId,
+      Iterable<? extends CorrelationId> alternateIds, RelNode r) {
+    return r.accept(
+        new DeduplicateCorrelateVariables(builder, canonicalId,
+            ImmutableSet.copyOf(alternateIds)));
+  }
+
+  @Override public RelNode visit(RelNode other) {
+    RelNode next = super.visit(other);
+    return next.accept(dedupRex);
+  }
+
   /**
    * Replaces alternative names of correlation variable to its canonical name.
    */
   private static class DeduplicateCorrelateVariablesShuttle extends RexShuttle {
     private final RexBuilder builder;
-    private final String canonical;
-    private final Set<String> altNames;
+    private final CorrelationId canonicalId;
+    private final ImmutableSet<CorrelationId> alternateIds;
+    private final DeduplicateCorrelateVariables shuttle;
 
-    public DeduplicateCorrelateVariablesShuttle(RexBuilder builder,
-        String canonical, Set<String> altNames) {
+    private DeduplicateCorrelateVariablesShuttle(RexBuilder builder,
+        CorrelationId canonicalId, ImmutableSet<CorrelationId> alternateIds,
+        DeduplicateCorrelateVariables shuttle) {
       this.builder = builder;
-      this.canonical = canonical;
-      this.altNames = altNames;
+      this.canonicalId = canonicalId;
+      this.alternateIds = alternateIds;
+      this.shuttle = shuttle;
     }
 
     @Override public RexNode visitCorrelVariable(RexCorrelVariable variable) {
-      if (!altNames.contains(variable.getName())) {
+      if (!alternateIds.contains(variable.id)) {
         return variable;
       }
 
-      return builder.makeCorrel(variable.getType(), canonical);
+      return builder.makeCorrel(variable.getType(), canonicalId);
     }
-  }
-
-  public DeduplicateCorrelateVariables(RexBuilder builder,
-      String canonical, Set<String> altNames) {
-    dedupRex = new DeduplicateCorrelateVariablesShuttle(builder,
-        canonical, altNames);
-  }
 
-  @Override public RelNode visit(RelNode other) {
-    RelNode next = super.visit(other);
-    return next.accept(dedupRex);
+    @Override public RexNode visitSubQuery(RexSubQuery subQuery) {
+      if (shuttle != null) {
+        RelNode r = subQuery.rel.accept(shuttle); // look inside sub-queries
+        if (r != subQuery.rel) {
+          subQuery = subQuery.clone(r);
+        }
+      }
+      return super.visitSubQuery(subQuery);
+    }
   }
 }
 


[11/50] [abbrv] calcite git commit: Following [CALCITE-897], empty string for boolean properties means true

Posted by jh...@apache.org.
Following [CALCITE-897], empty string for boolean properties means true


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

Branch: refs/heads/branch-release
Commit: 77e6f49c680cd154195e9e66c19356f0a2251bad
Parents: 970a8ca
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Oct 8 14:19:33 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 core/src/main/java/org/apache/calcite/util/Util.java    | 12 +++++++++++-
 .../test/java/org/apache/calcite/test/CalciteSuite.java |  2 +-
 .../java/org/apache/calcite/test/MongoAdapterIT.java    |  2 +-
 .../org/apache/calcite/adapter/tpcds/TpcdsTest.java     |  2 +-
 .../java/org/apache/calcite/adapter/tpch/TpchTest.java  |  2 +-
 site/_docs/history.md                                   |  2 +-
 site/_docs/howto.md                                     |  6 +++---
 .../java/org/apache/calcite/test/SplunkAdapterTest.java |  8 +++++---
 8 files changed, 24 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/77e6f49c/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index f7db6a4..35dc0ff 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -2293,8 +2293,18 @@ public class Util {
    * @return Whether property is true
    */
   public static boolean getBooleanProperty(String property) {
+    return getBooleanProperty(property, false);
+  }
+
+  /** Returns the value of a system property as a boolean, returning a given
+   * default value if the property is not specified. */
+  public static boolean getBooleanProperty(String property,
+      boolean defaultValue) {
     final String v = System.getProperties().getProperty(property);
-    return v != null && ("".equals(v) || "true".equalsIgnoreCase(v));
+    if (v == null) {
+      return defaultValue;
+    }
+    return "".equals(v) || "true".equalsIgnoreCase(v);
   }
 
   //~ Inner Classes ----------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/77e6f49c/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index 0ca057a..d9c9e4a 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -141,7 +141,7 @@ import org.junit.runners.Suite;
     PartiallyOrderedSetTest.class,
 
     // system tests and benchmarks (very slow, but usually only run if
-    // '-Dcalcite.test.slow=true' is specified)
+    // '-Dcalcite.test.slow' is specified)
     FoodmartTest.class
 })
 public class CalciteSuite {

http://git-wip-us.apache.org/repos/asf/calcite/blob/77e6f49c/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
----------------------------------------------------------------------
diff --git a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
index 81628c5..0dc3df1 100644
--- a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
+++ b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
@@ -113,7 +113,7 @@ public class MongoAdapterIT {
    * included if "it" profile is activated ({@code -Pit}). To disable,
    * specify {@code -Dcalcite.test.mongodb=false} on the Java command line. */
   public static final boolean ENABLED =
-      Boolean.valueOf(System.getProperty("calcite.test.mongodb", "true"));
+      Util.getBooleanProperty("calcite.test.mongodb", true);
 
   /** Whether to run this test. */
   protected boolean enabled() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/77e6f49c/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java
----------------------------------------------------------------------
diff --git a/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java b/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java
index 15fc496..b7f1f1c 100644
--- a/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java
+++ b/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java
@@ -47,7 +47,7 @@ import java.util.Random;
 
 /** Unit test for {@link org.apache.calcite.adapter.tpcds.TpcdsSchema}.
  *
- * <p>Only runs if {@code -Dcalcite.test.slow=true} is specified on the
+ * <p>Only runs if {@code -Dcalcite.test.slow} is specified on the
  * command-line.
  * (See {@link org.apache.calcite.test.CalciteAssert#ENABLE_SLOW}.)</p> */
 public class TpcdsTest {

http://git-wip-us.apache.org/repos/asf/calcite/blob/77e6f49c/plus/src/test/java/org/apache/calcite/adapter/tpch/TpchTest.java
----------------------------------------------------------------------
diff --git a/plus/src/test/java/org/apache/calcite/adapter/tpch/TpchTest.java b/plus/src/test/java/org/apache/calcite/adapter/tpch/TpchTest.java
index a7eb58b..0170520 100644
--- a/plus/src/test/java/org/apache/calcite/adapter/tpch/TpchTest.java
+++ b/plus/src/test/java/org/apache/calcite/adapter/tpch/TpchTest.java
@@ -37,7 +37,7 @@ import static org.junit.Assert.assertThat;
  *
  * <p>Because the TPC-H data generator takes time and memory to instantiate,
  * tests that read data (that is, most tests) only run
- * if {@code -Dcalcite.test.slow=true} is specified on the command-line.
+ * if {@code -Dcalcite.test.slow} is specified on the command-line.
  * (See {@link org.apache.calcite.test.CalciteAssert#ENABLE_SLOW}.)</p> */
 public class TpchTest {
   public static final String JAVA_VERSION =

http://git-wip-us.apache.org/repos/asf/calcite/blob/77e6f49c/site/_docs/history.md
----------------------------------------------------------------------
diff --git a/site/_docs/history.md b/site/_docs/history.md
index 00a4cbb..6769758 100644
--- a/site/_docs/history.md
+++ b/site/_docs/history.md
@@ -286,7 +286,7 @@ Bug fixes, API changes and minor enhancements
 {: #v1-4-0}
 
 In addition to a large number of bug fixes and minor enhancements,
-this release includes improvements to lattices and matierlized views,
+this release includes improvements to lattices and materialized views,
 and adds a builder API so that you can easily create relational
 algebra expressions.
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/77e6f49c/site/_docs/howto.md
----------------------------------------------------------------------
diff --git a/site/_docs/howto.md b/site/_docs/howto.md
index 3a03364..2f2bddc 100644
--- a/site/_docs/howto.md
+++ b/site/_docs/howto.md
@@ -93,7 +93,7 @@ environment, as follows.
 * `-Dcalcite.test.slow` enables tests that take longer to execute. For
   example, there are tests that create virtual TPC-H and TPC-DS schemas
   in-memory and run tests from those benchmarks.
-* `-Dcalcite.test.splunk=true` enables tests that run against Splunk.
+* `-Dcalcite.test.splunk` enables tests that run against Splunk.
   Splunk must be installed and running.
 
 ## Running integration tests
@@ -417,8 +417,8 @@ Before you start:
 * Check that `README`, `README.md` and `doc/howto.md` have the correct version number.
 * Set `version.major` and `version.minor` in `pom.xml`.
 * Make sure build and tests succeed, including with
-  -Dcalcite.test.db={mysql,hsqldb}, -Dcalcite.test.slow=true,
-  -Dcalcite.test.mongodb=true, -Dcalcite.test.splunk=true.
+  -Dcalcite.test.db={mysql,hsqldb}, -Dcalcite.test.slow,
+  -Dcalcite.test.mongodb, -Dcalcite.test.splunk.
 * Trigger a
   <a href="https://scan.coverity.com/projects/2966">Coverity scan</a>
   by merging the latest code into the `julianhyde/coverity_scan` branch,

http://git-wip-us.apache.org/repos/asf/calcite/blob/77e6f49c/splunk/src/test/java/org/apache/calcite/test/SplunkAdapterTest.java
----------------------------------------------------------------------
diff --git a/splunk/src/test/java/org/apache/calcite/test/SplunkAdapterTest.java b/splunk/src/test/java/org/apache/calcite/test/SplunkAdapterTest.java
index ce71d76..f67821d 100644
--- a/splunk/src/test/java/org/apache/calcite/test/SplunkAdapterTest.java
+++ b/splunk/src/test/java/org/apache/calcite/test/SplunkAdapterTest.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.util.Util;
+
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableSet;
 
@@ -46,9 +48,9 @@ public class SplunkAdapterTest {
 
   /** Whether to run Splunk tests. Disabled by default, because we do not expect
    * Splunk to be installed and populated data set. To enable,
-   * specify {@code -Dcalcite.test.splunk=true} on the Java command line. */
+   * specify {@code -Dcalcite.test.splunk} on the Java command line. */
   public static final boolean ENABLED =
-      Boolean.getBoolean("calcite.test.splunk");
+      Util.getBooleanProperty("calcite.test.splunk");
 
   /** Whether this test is enabled. Tests are disabled unless we know that
    * Splunk is present and loaded with the requisite data. */
@@ -240,7 +242,7 @@ public class SplunkAdapterTest {
         + "from \"splunk\".\"splunk\"",
         new Function<ResultSet, Void>() {
           public Void apply(ResultSet a0) {
-            final Set<String> actual = new HashSet<String>();
+            final Set<String> actual = new HashSet<>();
             try {
               while (a0.next()) {
                 actual.add(a0.getString(1));


[44/50] [abbrv] calcite git commit: [CALCITE-1058] Add method RelBuilder.empty, and rewrite LIMIT 0 to it

Posted by jh...@apache.org.
[CALCITE-1058] Add method RelBuilder.empty, and rewrite LIMIT 0 to it


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

Branch: refs/heads/branch-release
Commit: ee283cadf17d8add8e8ea2bf1f8dd5735f05dc1d
Parents: 3cba705
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Jan 13 21:37:27 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Jan 14 13:51:44 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/tools/RelBuilder.java    | 21 +++++++++++++
 .../org/apache/calcite/test/RelBuilderTest.java | 33 ++++++++++++++++++++
 2 files changed, 54 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/ee283cad/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 00ce6d1..cf9fcb6 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -1155,6 +1155,24 @@ public class RelBuilder {
     return true;
   }
 
+  /** Creates a relational expression that reads from an input and throws
+   * all of the rows away.
+   *
+   * <p>Note that this method always pops one relational expression from the
+   * stack. {@code values}, in contrast, does not pop any relational
+   * expressions, and always produces a leaf.
+   *
+   * <p>The default implementation creates a {@link Values} with the same
+   * specified row type as the input, and ignores the input entirely.
+   * But schema-on-query systems such as Drill might override this method to
+   * create a relation expression that retains the input, just to read its
+   * schema.
+   */
+  public RelBuilder empty() {
+    final Frame frame = Stacks.pop(stack);
+    return values(frame.rel.getRowType());
+  }
+
   /** Creates a {@link Values} with a specified row type.
    *
    * <p>This method can handle cases that {@link #values(String[], Object...)}
@@ -1274,6 +1292,9 @@ public class RelBuilder {
     }
     final RexNode offsetNode = offset <= 0 ? null : literal(offset);
     final RexNode fetchNode = fetch < 0 ? null : literal(fetch);
+    if (offsetNode == null && fetch == 0) {
+      return empty();
+    }
     if (offsetNode == null && fetchNode == null && fieldCollations.isEmpty()) {
       return this; // sort is trivial
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/ee283cad/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 0b232f4..db13a02 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -873,6 +873,25 @@ public class RelBuilderTest {
     assertThat(str(root), is(expected));
   }
 
+  @Test public void testEmpty() {
+    // Equivalent SQL:
+    //   SELECT deptno, true FROM dept LIMIT 0
+    // optimized to
+    //   VALUES
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("DEPT")
+            .project(builder.field(0), builder.literal(false))
+            .empty()
+            .build();
+    final String expected =
+        "LogicalValues(tuples=[[]])\n";
+    assertThat(str(root), is(expected));
+    final String expectedType =
+        "RecordType(TINYINT NOT NULL DEPTNO, BOOLEAN NOT NULL $f1) NOT NULL";
+    assertThat(root.getRowType().getFullTypeString(), is(expectedType));
+  }
+
   @Test public void testValues() {
     // Equivalent SQL:
     //   VALUES (true, 1), (false, -50) AS t(a, b)
@@ -1071,6 +1090,20 @@ public class RelBuilderTest {
     assertThat(str(root), is(expected));
   }
 
+  @Test public void testSortLimit0() {
+    // Equivalent SQL:
+    //   SELECT *
+    //   FROM emp
+    //   ORDER BY deptno DESC FETCH 0
+    final RelBuilder builder = RelBuilder.create(config().build());
+    final RelNode root =
+        builder.scan("EMP")
+            .sortLimit(-1, 0, builder.desc(builder.field("DEPTNO")))
+            .build();
+    final String expected = "LogicalValues(tuples=[[]])\n";
+    assertThat(str(root), is(expected));
+  }
+
   /** Tests that a sort on a field followed by a limit gives the same
    * effect as calling sortLimit.
    *


[03/50] [abbrv] calcite git commit: [CALCITE-1041] User-defined function that returns DATE or TIMESTAMP value

Posted by jh...@apache.org.
[CALCITE-1041] User-defined function that returns DATE or TIMESTAMP value

Rationalize code that translates to an from internal representation (e.g. DATE stored as int).

Fix an bug with overloaded UDFs, and a bug with UDF args of type SMALLINT (short).

Deprecate Utilities.equal.

Add methods to in BuiltInMethod.


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

Branch: refs/heads/branch-release
Commit: d4bbf58d6e5f4f67ebae412b5a1aae23769eccb7
Parents: efb668b
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jan 5 15:15:51 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/EnumUtils.java   | 118 ++++++++++++++++++-
 .../enumerable/EnumerableRelImplementor.java    |  19 +--
 .../adapter/enumerable/PhysTypeImpl.java        |  15 +--
 .../ReflectiveCallNotNullImplementor.java       |  36 +-----
 .../calcite/adapter/enumerable/RexImpTable.java |   7 +-
 .../adapter/enumerable/RexToLixTranslator.java  |  39 ++++--
 .../enumerable/StrictAggImplementor.java        |  22 ++--
 .../calcite/prepare/CalciteCatalogReader.java   |  48 +++++---
 .../org/apache/calcite/runtime/FlatLists.java   |  11 +-
 .../apache/calcite/runtime/SqlFunctions.java    |  37 ++++--
 .../org/apache/calcite/runtime/Utilities.java   |   2 +
 .../java/org/apache/calcite/sql/SqlUtil.java    |   2 +-
 .../sql/type/SqlTypeExplicitPrecedenceList.java |   1 +
 .../apache/calcite/sql/type/SqlTypeUtil.java    |   4 +
 .../org/apache/calcite/util/BuiltInMethod.java  |  16 ++-
 .../org/apache/calcite/test/CalciteAssert.java  |   3 +-
 .../java/org/apache/calcite/test/UdfTest.java   |  62 ++++++++++
 .../java/org/apache/calcite/util/Smalls.java    |  35 ++++--
 18 files changed, 357 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
index 17e0bf5..60408c6 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
@@ -20,6 +20,7 @@ import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Function2;
 import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.MethodDeclaration;
@@ -29,8 +30,11 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -44,6 +48,14 @@ import java.util.List;
  * style.
  */
 public class EnumUtils {
+
+  private static final Function<RexNode, Type> REX_TO_INTERNAL_TYPE =
+      new Function<RexNode, Type>() {
+        public Type apply(RexNode node) {
+          return toInternal(node.getType());
+        }
+      };
+
   private EnumUtils() {}
 
   static final boolean BRIDGE_METHODS = true;
@@ -54,7 +66,8 @@ public class EnumUtils {
   static final List<Expression> NO_EXPRS =
       ImmutableList.of();
 
-  public static final String[] LEFT_RIGHT = {"left", "right"};
+  public static final List<String> LEFT_RIGHT =
+      ImmutableList.of("left", "right");
 
   /** Declares a method that overrides another method. */
   public static MethodDeclaration overridingMethodDecl(Method method,
@@ -117,12 +130,10 @@ public class EnumUtils {
   static Expression joinSelector(JoinRelType joinType, PhysType physType,
       List<PhysType> inputPhysTypes) {
     // A parameter for each input.
-    final List<ParameterExpression> parameters =
-        new ArrayList<ParameterExpression>();
+    final List<ParameterExpression> parameters = new ArrayList<>();
 
     // Generate all fields.
-    final List<Expression> expressions =
-        new ArrayList<Expression>();
+    final List<Expression> expressions = new ArrayList<>();
     final int outputFieldCount = physType.getRowType().getFieldCount();
     for (Ord<PhysType> ord : Ord.zip(inputPhysTypes)) {
       final PhysType inputPhysType =
@@ -132,7 +143,7 @@ public class EnumUtils {
       // Function<T> always operates on boxed arguments
       final ParameterExpression parameter =
           Expressions.parameter(Primitive.box(inputPhysType.getJavaRowType()),
-              EnumUtils.LEFT_RIGHT[ord.i]);
+              EnumUtils.LEFT_RIGHT.get(ord.i));
       parameters.add(parameter);
       if (expressions.size() == outputFieldCount) {
         // For instance, if semi-join needs to return just the left inputs
@@ -158,6 +169,101 @@ public class EnumUtils {
         physType.record(expressions),
         parameters);
   }
+
+  /** Converts from internal representation to JDBC representation used by
+   * arguments of user-defined functions. For example, converts date values from
+   * {@code int} to {@link java.sql.Date}. */
+  static Expression fromInternal(Expression e, Class<?> targetType) {
+    if (e == ConstantUntypedNull.INSTANCE) {
+      return e;
+    }
+    if (!(e.getType() instanceof Class)) {
+      return e;
+    }
+    if (targetType.isAssignableFrom((Class) e.getType())) {
+      return e;
+    }
+    if (targetType == java.sql.Date.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_DATE.method, e);
+    }
+    if (targetType == java.sql.Time.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIME.method, e);
+    }
+    if (targetType == java.sql.Timestamp.class) {
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method, e);
+    }
+    if (Primitive.is(e.type)
+        && Primitive.isBox(targetType)) {
+      // E.g. e is "int", target is "Long", generate "(long) e".
+      return Expressions.convert_(e,
+          Primitive.ofBox(targetType).primitiveClass);
+    }
+    return e;
+  }
+
+  static List<Expression> fromInternal(Class<?>[] targetTypes,
+      List<Expression> expressions) {
+    final List<Expression> list = new ArrayList<>();
+    for (int i = 0; i < expressions.size(); i++) {
+      list.add(fromInternal(expressions.get(i), targetTypes[i]));
+    }
+    return list;
+  }
+
+  static Type fromInternal(Type type) {
+    if (type == java.sql.Date.class || type == java.sql.Time.class) {
+      return int.class;
+    }
+    if (type == java.sql.Timestamp.class) {
+      return long.class;
+    }
+    return type;
+  }
+
+  static Type toInternal(RelDataType type) {
+    switch (type.getSqlTypeName()) {
+    case DATE:
+    case TIME:
+      return type.isNullable() ? Integer.class : int.class;
+    case TIMESTAMP:
+      return type.isNullable() ? Long.class : long.class;
+    default:
+      return null; // we don't care; use the default storage type
+    }
+  }
+
+  static List<Type> internalTypes(List<? extends RexNode> operandList) {
+    return Lists.transform(operandList, REX_TO_INTERNAL_TYPE);
+  }
+
+  static Expression enforce(final Type storageType,
+      final Expression e) {
+    if (storageType != null && e.type != storageType) {
+      if (e.type == java.sql.Date.class) {
+        if (storageType == int.class) {
+          return Expressions.call(BuiltInMethod.DATE_TO_INT.method, e);
+        }
+        if (storageType == Integer.class) {
+          return Expressions.call(BuiltInMethod.DATE_TO_INT_OPTIONAL.method, e);
+        }
+      } else if (e.type == java.sql.Time.class) {
+        if (storageType == int.class) {
+          return Expressions.call(BuiltInMethod.TIME_TO_INT.method, e);
+        }
+        if (storageType == Integer.class) {
+          return Expressions.call(BuiltInMethod.TIME_TO_INT_OPTIONAL.method, e);
+        }
+      } else if (e.type == java.sql.Timestamp.class) {
+        if (storageType == long.class) {
+          return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG.method, e);
+        }
+        if (storageType == Long.class) {
+          return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG_OPTIONAL.method, e);
+        }
+      }
+    }
+    return e;
+  }
 }
 
 // End EnumUtils.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
index 1c446a3..bd77f2b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
@@ -39,7 +39,6 @@ import org.apache.calcite.linq4j.tree.Types;
 import org.apache.calcite.linq4j.tree.Visitor;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.runtime.Bindable;
-import org.apache.calcite.runtime.Utilities;
 import org.apache.calcite.util.BuiltInMethod;
 
 import com.google.common.base.Function;
@@ -49,6 +48,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 
 import java.io.Serializable;
+import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
@@ -239,9 +239,7 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
               ? Expressions.equal(
                   Expressions.field(thisParameter, field.getName()),
                   Expressions.field(thatParameter, field.getName()))
-              : Expressions.call(
-                  Utilities.class,
-                  "equal",
+              : Expressions.call(BuiltInMethod.OBJECTS_EQUAL.method,
                   Expressions.field(thisParameter, field.getName()),
                   Expressions.field(thatParameter, field.getName())));
     }
@@ -270,13 +268,14 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
     blockBuilder3.add(
         Expressions.declare(0, hParameter, constantZero));
     for (Types.RecordField field : type.getRecordFields()) {
+      final Method method = BuiltInMethod.HASH.method;
       blockBuilder3.add(
           Expressions.statement(
               Expressions.assign(
                   hParameter,
                   Expressions.call(
-                      Utilities.class,
-                      "hash",
+                      method.getDeclaringClass(),
+                      method.getName(),
                       ImmutableList.of(
                           hParameter,
                           Expressions.field(thisParameter, field))))));
@@ -312,9 +311,11 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
     for (Types.RecordField field : type.getRecordFields()) {
       MethodCallExpression compareCall;
       try {
-        compareCall = Expressions.call(
-            Utilities.class,
-            field.nullable() ? "compareNullsLast" : "compare",
+        final Method method = (field.nullable()
+            ? BuiltInMethod.COMPARE_NULLS_LAST
+            : BuiltInMethod.COMPARE).method;
+        compareCall = Expressions.call(method.getDeclaringClass(),
+            method.getName(),
             Expressions.field(thisParameter, field),
             Expressions.field(thatParameter, field));
       } catch (RuntimeException e) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
index af42c16..918833e 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
@@ -40,6 +40,7 @@ import org.apache.calcite.util.Util;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
 import java.util.AbstractList;
@@ -301,17 +302,17 @@ public class PhysTypeImpl implements PhysType {
       final boolean descending =
           collation.getDirection()
               == RelFieldCollation.Direction.DESCENDING;
+      final Method method = (fieldNullable(index)
+          ? (nullsFirst ^ descending
+              ? BuiltInMethod.COMPARE_NULLS_FIRST
+              : BuiltInMethod.COMPARE_NULLS_LAST)
+          : BuiltInMethod.COMPARE).method;
       body.add(
           Expressions.statement(
               Expressions.assign(
                   parameterC,
-                  Expressions.call(
-                      Utilities.class,
-                      fieldNullable(index)
-                          ? (nullsFirst ^ descending
-                              ? "compareNullsFirst"
-                              : "compareNullsLast")
-                          : "compare",
+                  Expressions.call(method.getDeclaringClass(),
+                      method.getName(),
                       arg0,
                       arg1))));
       body.add(

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
index 6414e03..ee2ad1f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
@@ -16,16 +16,13 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
-import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.NewExpression;
 import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.runtime.SqlFunctions;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -49,7 +46,8 @@ public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
 
   public Expression implement(RexToLixTranslator translator,
       RexCall call, List<Expression> translatedOperands) {
-    translatedOperands = fromInternal(translatedOperands);
+    translatedOperands =
+        EnumUtils.fromInternal(method.getParameterTypes(), translatedOperands);
     if ((method.getModifiers() & Modifier.STATIC) != 0) {
       return Expressions.call(method, translatedOperands);
     } else {
@@ -61,36 +59,6 @@ public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
     }
   }
 
-  protected List<Expression> fromInternal(List<Expression> expressions) {
-    final List<Expression> list = new ArrayList<>();
-    final Class[] types = method.getParameterTypes();
-    for (int i = 0; i < expressions.size(); i++) {
-      list.add(fromInternal(expressions.get(i), types[i]));
-    }
-    return list;
-  }
-
-  protected Expression fromInternal(Expression e, Class<?> targetType) {
-    if (e == ConstantUntypedNull.INSTANCE) {
-      return e;
-    }
-    if (!(e.getType() instanceof Class)) {
-      return e;
-    }
-    if (targetType.isAssignableFrom((Class) e.getType())) {
-      return e;
-    }
-    if (targetType == java.sql.Date.class) {
-      return Expressions.call(SqlFunctions.class, "internalToDate", e);
-    }
-    if (targetType == java.sql.Time.class) {
-      return Expressions.call(SqlFunctions.class, "internalToTime", e);
-    }
-    if (targetType == java.sql.Timestamp.class) {
-      return Expressions.call(SqlFunctions.class, "internalToTimestamp", e);
-    }
-    return e;
-  }
 }
 
 // End ReflectiveCallNotNullImplementor.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 32c043c..20872b6 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
@@ -1042,9 +1042,12 @@ public class RexImpTable {
       Expression acc = add.accumulator().get(0);
       Expression arg = add.arguments().get(0);
       SqlAggFunction aggregation = info.aggregation();
+      final Method method = (aggregation == MIN
+          ? BuiltInMethod.LESSER
+          : BuiltInMethod.GREATER).method;
       Expression next = Expressions.call(
-          SqlFunctions.class,
-          aggregation == MIN ? "lesser" : "greater",
+          method.getDeclaringClass(),
+          method.getName(),
           acc,
           Expressions.unbox(arg));
       accAdvance(add, acc, next);

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index eed3d9d..2af6b46 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -181,7 +181,7 @@ public class RexToLixTranslator {
     List<Type> storageTypes = null;
     if (outputPhysType != null) {
       final RelDataType rowType = outputPhysType.getRowType();
-      storageTypes = new ArrayList<Type>(rowType.getFieldCount());
+      storageTypes = new ArrayList<>(rowType.getFieldCount());
       for (int i = 0; i < rowType.getFieldCount(); i++) {
         storageTypes.add(outputPhysType.getJavaFieldType(i));
       }
@@ -217,6 +217,7 @@ public class RexToLixTranslator {
   Expression translate(RexNode expr, RexImpTable.NullAs nullAs,
       Type storageType) {
     Expression expression = translate0(expr, nullAs, storageType);
+    expression = EnumUtils.enforce(storageType, expression);
     assert expression != null;
     return list.append("v", expression);
   }
@@ -500,9 +501,7 @@ public class RexToLixTranslator {
       }
       InputGetter getter =
           correlates.apply(((RexCorrelVariable) target).getName());
-      Expression res =
-          getter.field(list, fieldAccess.getField().getIndex(), storageType);
-      return res;
+      return getter.field(list, fieldAccess.getField().getIndex(), storageType);
     default:
       if (expr instanceof RexCall) {
         return translateCall((RexCall) expr, nullAs);
@@ -643,9 +642,17 @@ public class RexToLixTranslator {
   public List<Expression> translateList(
       List<RexNode> operandList,
       RexImpTable.NullAs nullAs) {
-    final List<Expression> list = new ArrayList<Expression>();
-    for (RexNode rex : operandList) {
-      list.add(translate(rex, nullAs));
+    return translateList(operandList, nullAs,
+        EnumUtils.internalTypes(operandList));
+  }
+
+  public List<Expression> translateList(
+      List<RexNode> operandList,
+      RexImpTable.NullAs nullAs,
+      List<? extends Type> storageTypes) {
+    final List<Expression> list = new ArrayList<>();
+    for (Pair<RexNode, ? extends Type> e : Pair.zip(operandList, storageTypes)) {
+      list.add(translate(e.left, nullAs, e.right));
     }
     return list;
   }
@@ -663,7 +670,7 @@ public class RexToLixTranslator {
    * @return translated expressions
    */
   public List<Expression> translateList(List<? extends RexNode> operandList) {
-    return translateList(operandList, null);
+    return translateList(operandList, EnumUtils.internalTypes(operandList));
   }
 
   /**
@@ -682,7 +689,7 @@ public class RexToLixTranslator {
    */
   public List<Expression> translateList(List<? extends RexNode> operandList,
       List<? extends Type> storageTypes) {
-    final List<Expression> list = new ArrayList<Expression>(operandList.size());
+    final List<Expression> list = new ArrayList<>(operandList.size());
 
     for (int i = 0; i < operandList.size(); i++) {
       RexNode rex = operandList.get(i);
@@ -812,6 +819,19 @@ public class RexToLixTranslator {
         }
       }
       return Expressions.box(operand, toBox);
+    } else if (toType == java.sql.Date.class) {
+      // E.g. from "int" or "Integer" to "java.sql.Date",
+      // generate "SqlFunctions.internalToDate".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_DATE.method, operand);
+    } else if (toType == java.sql.Time.class) {
+      // E.g. from "int" or "Integer" to "java.sql.Time",
+      // generate "SqlFunctions.internalToTime".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIME.method, operand);
+    } else if (toType == java.sql.Timestamp.class) {
+      // E.g. from "long" or "Long" to "java.sql.Timestamp",
+      // generate "SqlFunctions.internalToTimestamp".
+      return Expressions.call(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method,
+          operand);
     } else if (toType == BigDecimal.class) {
       if (fromBox != null) {
         // E.g. from "Integer" to "BigDecimal".
@@ -1048,6 +1068,7 @@ public class RexToLixTranslator {
    * it is not null. It is easier to throw (and caller will always handle)
    * than to check exhaustively beforehand. */
   static class AlwaysNull extends ControlFlowException {
+    @SuppressWarnings("ThrowableInstanceNeverThrown")
     public static final AlwaysNull INSTANCE = new AlwaysNull();
 
     private AlwaysNull() {}

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
index 8b463e8..a34813c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/StrictAggImplementor.java
@@ -65,13 +65,7 @@ public abstract class StrictAggImplementor implements AggImplementor {
     if (!needTrackEmptySet) {
       return subState;
     }
-    boolean hasNullableArgs = false;
-    for (RelDataType type : info.parameterRelTypes()) {
-      if (type.isNullable()) {
-        hasNullableArgs = true;
-        break;
-      }
-    }
+    final boolean hasNullableArgs = anyNullable(info.parameterRelTypes());
     trackNullsPerRow = !(info instanceof WinAggContext) || hasNullableArgs;
 
     List<Type> res = new ArrayList<>(subState.size() + 1);
@@ -80,8 +74,20 @@ public abstract class StrictAggImplementor implements AggImplementor {
     return res;
   }
 
+  private boolean anyNullable(List<? extends RelDataType> types) {
+    for (RelDataType type : types) {
+      if (type.isNullable()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   public List<Type> getNotNullState(AggContext info) {
-    return Collections.singletonList(Primitive.unbox(info.returnType()));
+    Type type = info.returnType();
+    type = EnumUtils.fromInternal(type);
+    type = Primitive.unbox(type);
+    return Collections.singletonList(type);
   }
 
   public final void implementReset(AggContext info, AggResetContext reset) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 e3211ea..68b297e 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -27,18 +27,19 @@ import org.apache.calcite.schema.AggregateFunction;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.ScalarFunction;
-import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.sql.type.FamilyOperandTypeChecker;
 import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlMoniker;
@@ -263,14 +264,11 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
         OperandTypes.family(typeFamilies, optional);
     final List<RelDataType> paramTypes = toSql(argTypes);
     if (function instanceof ScalarFunction) {
-      return new SqlUserDefinedFunction(name,
-          ReturnTypes.explicit(Schemas.proto((ScalarFunction) function)),
+      return new SqlUserDefinedFunction(name, infer((ScalarFunction) function),
           InferTypes.explicit(argTypes), typeChecker, paramTypes, function);
     } else if (function instanceof AggregateFunction) {
-      final RelDataType returnType =
-          ((AggregateFunction) function).getReturnType(typeFactory);
       return new SqlUserDefinedAggFunction(name,
-          ReturnTypes.explicit(returnType), InferTypes.explicit(argTypes),
+          infer((AggregateFunction) function), InferTypes.explicit(argTypes),
           typeChecker, (AggregateFunction) function);
     } else if (function instanceof TableMacro) {
       return new SqlUserDefinedTableMacro(name, ReturnTypes.CURSOR,
@@ -285,21 +283,43 @@ 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);
+        return toSql(type);
+      }
+    };
+  }
+
+  private SqlReturnTypeInference infer(final AggregateFunction function) {
+    return new SqlReturnTypeInference() {
+      public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+        final RelDataType type = function.getReturnType(typeFactory);
+        return toSql(type);
+      }
+    };
+  }
+
   private List<RelDataType> toSql(List<RelDataType> types) {
     return Lists.transform(types,
         new com.google.common.base.Function<RelDataType, RelDataType>() {
-          public RelDataType apply(RelDataType input) {
-            if (input instanceof RelDataTypeFactoryImpl.JavaType
-                && ((RelDataTypeFactoryImpl.JavaType) input).getJavaClass()
-                == Object.class) {
-              return typeFactory.createTypeWithNullability(
-                  typeFactory.createSqlType(SqlTypeName.ANY), true);
-            }
-            return typeFactory.toSql(input);
+          public RelDataType apply(RelDataType type) {
+            return toSql(type);
           }
         });
   }
 
+  private RelDataType toSql(RelDataType type) {
+    if (type instanceof RelDataTypeFactoryImpl.JavaType
+        && ((RelDataTypeFactoryImpl.JavaType) type).getJavaClass()
+        == Object.class) {
+      return typeFactory.createTypeWithNullability(
+          typeFactory.createSqlType(SqlTypeName.ANY), true);
+    }
+    return typeFactory.toSql(type);
+  }
+
   public List<SqlOperator> getOperatorList() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
index 4a001cf..f3cfae4 100644
--- a/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
+++ b/core/src/main/java/org/apache/calcite/runtime/FlatLists.java
@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.ListIterator;
+import java.util.Objects;
 import java.util.RandomAccess;
 
 /**
@@ -261,8 +262,8 @@ public class FlatLists {
       }
       if (o instanceof Flat2List) {
         Flat2List that = (Flat2List) o;
-        return Utilities.equal(this.t0, that.t0)
-            && Utilities.equal(this.t1, that.t1);
+        return Objects.equals(this.t0, that.t0)
+            && Objects.equals(this.t1, that.t1);
       }
       return Arrays.asList(t0, t1).equals(o);
     }
@@ -386,9 +387,9 @@ public class FlatLists {
       }
       if (o instanceof Flat3List) {
         Flat3List that = (Flat3List) o;
-        return Utilities.equal(this.t0, that.t0)
-            && Utilities.equal(this.t1, that.t1)
-            && Utilities.equal(this.t2, that.t2);
+        return Objects.equals(this.t0, that.t0)
+            && Objects.equals(this.t1, that.t1)
+            && Objects.equals(this.t2, that.t2);
       }
       return o.equals(this);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 22b0aaa..8abdb97 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -1040,6 +1040,10 @@ public class SqlFunctions {
         : (Short) cannotConvert(o, short.class);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL DATE type
+   * ({@link java.sql.Date}) to internal representation (int).
+   *
+   * <p>Converse of {@link #internalToDate(int)}. */
   public static int toInt(java.util.Date v) {
     return toInt(v, LOCAL_TZ);
   }
@@ -1062,6 +1066,10 @@ public class SqlFunctions {
     return toLong(v, LOCAL_TZ);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL TIME type
+   * ({@link java.sql.Time}) to internal representation (int).
+   *
+   * <p>Converse of {@link #internalToTime(int)}. */
   public static int toInt(java.sql.Time v) {
     return (int) (toLong(v) % DateTimeUtils.MILLIS_PER_DAY);
   }
@@ -1085,6 +1093,10 @@ public class SqlFunctions {
         : (Integer) cannotConvert(o, int.class);
   }
 
+  /** Converts the Java type used for UDF parameters of SQL TIMESTAMP type
+   * ({@link java.sql.Timestamp}) to internal representation (long).
+   *
+   * <p>Converse of {@link #internalToTimestamp(long)}. */
   public static long toLong(Timestamp v) {
     return toLong(v, LOCAL_TZ);
   }
@@ -1175,33 +1187,34 @@ public class SqlFunctions {
 
   /** Converts the internal representation of a SQL DATE (int) to the Java
    * type used for UDF parameters ({@link java.sql.Date}). */
-  public static java.sql.Date internalToDate(int x) {
-    return new java.sql.Date(x * DateTimeUtils.MILLIS_PER_DAY);
+  public static java.sql.Date internalToDate(int v) {
+    final long t = v * DateTimeUtils.MILLIS_PER_DAY;
+    return new java.sql.Date(t - LOCAL_TZ.getOffset(t));
   }
 
   /** As {@link #internalToDate(int)} but allows nulls. */
-  public static java.sql.Date internalToDate(Integer x) {
-    return x == null ? null : internalToDate(x.intValue());
+  public static java.sql.Date internalToDate(Integer v) {
+    return v == null ? null : internalToDate(v.intValue());
   }
 
   /** Converts the internal representation of a SQL TIME (int) to the Java
    * type used for UDF parameters ({@link java.sql.Time}). */
-  public static java.sql.Time internalToTime(int x) {
-    return new java.sql.Time(x);
+  public static java.sql.Time internalToTime(int v) {
+    return new java.sql.Time(v - LOCAL_TZ.getOffset(v));
   }
 
-  public static java.sql.Time internalToTime(Integer x) {
-    return x == null ? null : internalToTime(x.intValue());
+  public static java.sql.Time internalToTime(Integer v) {
+    return v == null ? null : internalToTime(v.intValue());
   }
 
   /** Converts the internal representation of a SQL TIMESTAMP (long) to the Java
    * type used for UDF parameters ({@link java.sql.Timestamp}). */
-  public static java.sql.Timestamp internalToTimestamp(long x) {
-    return new java.sql.Timestamp(x);
+  public static java.sql.Timestamp internalToTimestamp(long v) {
+    return new java.sql.Timestamp(v - LOCAL_TZ.getOffset(v));
   }
 
-  public static java.sql.Timestamp internalToTimestamp(Long x) {
-    return x == null ? null : internalToTimestamp(x.longValue());
+  public static java.sql.Timestamp internalToTimestamp(Long v) {
+    return v == null ? null : internalToTimestamp(v.longValue());
   }
 
   // Don't need shortValueOf etc. - Short.valueOf is sufficient.

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/runtime/Utilities.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Utilities.java b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
index 691afd5..cf4fc0c 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Utilities.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
@@ -29,6 +29,8 @@ public class Utilities {
   protected Utilities() {
   }
 
+  /** @deprecated Use {@link java.util.Objects#equals}. */
+  @Deprecated // to be removed before 2.0
   public static boolean equal(Object o0, Object o1) {
     // Same as java.lang.Objects.equals (JDK 1.7 and later)
     // and com.google.common.base.Objects.equal

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index 66fb911..87e73e0 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -493,7 +493,7 @@ public abstract class SqlUtil {
         final RelDataType argType = p.right;
         final RelDataType paramType = p.left;
         if (argType != null
-            && !SqlTypeUtil.canAssignFrom(paramType, argType)) {
+            && !SqlTypeUtil.canCastFrom(paramType, argType, false)) {
           iter.remove();
           continue loop;
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 1da123d..4fe315b 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
@@ -71,6 +71,7 @@ public class SqlTypeExplicitPrecedenceList
       ImmutableMap.<SqlTypeName, SqlTypeExplicitPrecedenceList>builder()
           .put(SqlTypeName.BOOLEAN, list(SqlTypeName.BOOLEAN))
           .put(SqlTypeName.TINYINT, numeric(SqlTypeName.TINYINT))
+          .put(SqlTypeName.SMALLINT, numeric(SqlTypeName.SMALLINT))
           .put(SqlTypeName.INTEGER, numeric(SqlTypeName.INTEGER))
           .put(SqlTypeName.BIGINT, numeric(SqlTypeName.BIGINT))
           .put(SqlTypeName.DECIMAL, numeric(SqlTypeName.DECIMAL))

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 658acd4..204b7ea 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -589,6 +589,7 @@ public abstract class SqlTypeUtil {
    * @return true if type has a representation as a Java primitive (ignoring
    * nullability)
    */
+  @Deprecated // to be removed before 2.0
   public static boolean isJavaPrimitive(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -614,6 +615,7 @@ public abstract class SqlTypeUtil {
   /**
    * @return class name of the wrapper for the primitive data type.
    */
+  @Deprecated // to be removed before 2.0
   public static String getPrimitiveWrapperJavaClassName(RelDataType type) {
     if (type == null) {
       return null;
@@ -627,6 +629,7 @@ public abstract class SqlTypeUtil {
     case BOOLEAN:
       return "Boolean";
     default:
+      //noinspection deprecation
       return getNumericJavaClassName(type);
     }
   }
@@ -634,6 +637,7 @@ public abstract class SqlTypeUtil {
   /**
    * @return class name of the numeric data type.
    */
+  @Deprecated // to be removed before 2.0
   public static String getNumericJavaClassName(RelDataType type) {
     if (type == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 4c7b41c..c899cb8 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -56,6 +56,7 @@ import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.runtime.ResultSetEnumerable;
 import org.apache.calcite.runtime.SortedMultiMap;
 import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.runtime.Utilities;
 import org.apache.calcite.schema.FilterableTable;
 import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.schema.ProjectableFilterableTable;
@@ -82,6 +83,7 @@ import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.TimeZone;
 import javax.sql.DataSource;
 
@@ -254,10 +256,15 @@ public enum BuiltInMethod {
   IS_TRUE(SqlFunctions.class, "isTrue", Boolean.class),
   IS_NOT_FALSE(SqlFunctions.class, "isNotFalse", Boolean.class),
   NOT(SqlFunctions.class, "not", Boolean.class),
+  LESSER(SqlFunctions.class, "lesser", Comparable.class, Comparable.class),
+  GREATER(SqlFunctions.class, "greater", Comparable.class, Comparable.class),
   MODIFIABLE_TABLE_GET_MODIFIABLE_COLLECTION(ModifiableTable.class,
       "getModifiableCollection"),
   SCANNABLE_TABLE_SCAN(ScannableTable.class, "scan", DataContext.class),
   STRING_TO_BOOLEAN(SqlFunctions.class, "toBoolean", String.class),
+  INTERNAL_TO_DATE(SqlFunctions.class, "internalToDate", int.class),
+  INTERNAL_TO_TIME(SqlFunctions.class, "internalToTime", int.class),
+  INTERNAL_TO_TIMESTAMP(SqlFunctions.class, "internalToTimestamp", long.class),
   STRING_TO_DATE(DateTimeUtils.class, "dateStringToUnixDate", String.class),
   STRING_TO_TIME(DateTimeUtils.class, "timeStringToUnixDate", String.class),
   STRING_TO_TIMESTAMP(DateTimeUtils.class, "timestampStringToUnixDate",
@@ -288,8 +295,13 @@ public enum BuiltInMethod {
   BOOLEAN_TO_STRING(SqlFunctions.class, "toString", boolean.class),
   JDBC_ARRAY_TO_LIST(SqlFunctions.class, "arrayToList", java.sql.Array.class),
   OBJECT_TO_STRING(Object.class, "toString"),
-  OBJECTS_EQUAL(com.google.common.base.Objects.class, "equal", Object.class,
-      Object.class),
+  OBJECTS_EQUAL(Objects.class, "equals", Object.class, Object.class),
+  HASH(Utilities.class, "hash", int.class, Object.class),
+  COMPARE(Utilities.class, "compare", Comparable.class, Comparable.class),
+  COMPARE_NULLS_FIRST(Utilities.class, "compareNullsFirst", Comparable.class,
+      Comparable.class),
+  COMPARE_NULLS_LAST(Utilities.class, "compareNullsLast", Comparable.class,
+      Comparable.class),
   ROUND_LONG(SqlFunctions.class, "round", long.class, long.class),
   ROUND_INT(SqlFunctions.class, "round", int.class, int.class),
   DATE_TO_INT(SqlFunctions.class, "toInt", java.util.Date.class),

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 f9e2bea..4b36b51 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -275,7 +275,8 @@ public class CalciteAssert {
             throw new AssertionError("expected 1 column");
           }
           final String resultString = resultSet.getString(1);
-          assertEquals(expected, Util.toLinux(resultString));
+          assertEquals(expected,
+              resultString == null ? null : Util.toLinux(resultString));
           return null;
         } catch (SQLException e) {
           throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 7d707fe..a8d7766 100644
--- a/core/src/test/java/org/apache/calcite/test/UdfTest.java
+++ b/core/src/test/java/org/apache/calcite/test/UdfTest.java
@@ -105,6 +105,12 @@ public class UdfTest {
         + "         {\n"
         + "           name: 'COUNT_ARGS',\n"
         + "           className: '"
+        + Smalls.CountArgs1NullableFunction.class.getName()
+        + "'\n"
+        + "         },\n"
+        + "         {\n"
+        + "           name: 'COUNT_ARGS',\n"
+        + "           className: '"
         + Smalls.CountArgs2Function.class.getName()
         + "'\n"
         + "         },\n"
@@ -261,6 +267,14 @@ public class UdfTest {
         .returns("P0=0; P1=1; P2=2\n");
   }
 
+  @Test public void testUdfOverloadedNullable() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("values (\"adhoc\".count_args(),\n"
+        + " \"adhoc\".count_args(cast(null as smallint)),\n"
+        + " \"adhoc\".count_args(0, 0))")
+        .returns("EXPR$0=0; EXPR$1=-1; EXPR$2=2\n");
+  }
+
   /** Tests passing parameters to user-defined function by name. */
   @Test public void testUdfArgumentName() {
     final CalciteAssert.AssertThat with = withUdf();
@@ -586,6 +600,54 @@ public class UdfTest {
     with.query("values \"adhoc\".\"timestampFun\"(cast(null as timestamp))")
         .returnsValue("-1");
   }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1041">[CALCITE-1041]
+   * User-defined function returns DATE or TIMESTAMP value</a>. */
+  @Test public void testReturnDate() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("values \"adhoc\".\"toDateFun\"(0)")
+        .returnsValue("1970-01-01");
+    with.query("values \"adhoc\".\"toDateFun\"(1)")
+        .returnsValue("1970-01-02");
+    with.query("values \"adhoc\".\"toDateFun\"(cast(null as bigint))")
+        .returnsValue(null);
+    with.query("values \"adhoc\".\"toTimeFun\"(0)")
+        .returnsValue("00:00:00");
+    with.query("values \"adhoc\".\"toTimeFun\"(90000)")
+        .returnsValue("00:01:30");
+    with.query("values \"adhoc\".\"toTimeFun\"(cast(null as bigint))")
+        .returnsValue(null);
+    with.query("values \"adhoc\".\"toTimestampFun\"(0)")
+        .returnsValue("1970-01-01 00:00:00");
+    with.query("values \"adhoc\".\"toTimestampFun\"(86490000)")
+        .returnsValue("1970-01-02 00:01:30");
+    with.query("values \"adhoc\".\"toTimestampFun\"(cast(null as bigint))")
+        .returnsValue(null);
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1041">[CALCITE-1041]
+   * User-defined function returns DATE or TIMESTAMP value</a>. */
+  @Test public void testReturnDate2() {
+    final CalciteAssert.AssertThat with = withUdf();
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  cast('1970-01-01 00:00:00' as timestamp),\n"
+        + "  cast('1997-02-01 00:00:00' as timestamp))")
+        .returnsValue("0");
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  timestamp '1970-01-01 00:00:00',\n"
+        + "  timestamp '1997-02-01 00:00:00')")
+        .returnsValue("0");
+    with.query("select * from (values 0) as t(c)\n"
+        + "where \"adhoc\".\"toTimestampFun\"(c) in (\n"
+        + "  '1970-01-01 00:00:00',\n"
+        + "  '1997-02-01 00:00:00')")
+        .returnsValue("0");
+  }
+
 }
 
 // End UdfTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d4bbf58d/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 62cc064..d9e07ef 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.SchemaPlus;
@@ -418,20 +419,34 @@ public class Smalls {
   public static class AllTypesFunction {
     private AllTypesFunction() {}
 
-    public static long dateFun(java.sql.Date x) { return x == null ? -1L : x.getTime(); }
-    public static long timestampFun(java.sql.Timestamp x) { return x == null ? -1L : x.getTime(); }
-    public static long timeFun(java.sql.Time x) { return x == null ? -1L : x.getTime(); }
+    // We use SqlFunctions.toLong(Date) ratter than Date.getTime(),
+    // and SqlFunctions.internalToTimestamp(long) rather than new Date(long),
+    // because the contract of JDBC (also used by UDFs) is to represent
+    // date-time values in the LOCAL time zone.
 
-    public static java.sql.Date toDateFun(int x) { return new java.sql.Date(x); }
+    public static long dateFun(java.sql.Date v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+    public static long timestampFun(java.sql.Timestamp v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+    public static long timeFun(java.sql.Time v) {
+      return v == null ? -1L : SqlFunctions.toLong(v);
+    }
+
+    /** Overloaded, in a challenging way, with {@link #toDateFun(Long)}. */
+    public static java.sql.Date toDateFun(int v) {
+      return SqlFunctions.internalToDate(v);
+    }
 
-    public static java.sql.Date toDateFun(Long x) {
-      return x == null ? null : new java.sql.Date(x);
+    public static java.sql.Date toDateFun(Long v) {
+      return v == null ? null : SqlFunctions.internalToDate(v.intValue());
     }
-    public static java.sql.Timestamp toTimestampFun(Long x) {
-      return x == null ? null : new java.sql.Timestamp(x);
+    public static java.sql.Timestamp toTimestampFun(Long v) {
+      return SqlFunctions.internalToTimestamp(v);
     }
-    public static java.sql.Time toTimeFun(Long x) {
-      return x == null ? null : new java.sql.Time(x);
+    public static java.sql.Time toTimeFun(Long v) {
+      return v == null ? null : SqlFunctions.internalToTime(v.intValue());
     }
   }
 


[26/50] [abbrv] calcite git commit: [CALCITE-816] Represent sub-query as a RexNode

Posted by jh...@apache.org.
[CALCITE-816] Represent sub-query as a RexNode

Reduce 3-value logic to 1- or 2-value logic.

Optimize certain IN and EXISTS to an inner join.

Represent correlation variables using CorrelationId wherever possible.

In Join, replace field "ImmutableSet<String> variablesStopped" with
"ImmutableSet<CorrelationId> variablesSet". RelNode.getVariablesSet
is now preferred to RelNode.getVariablesStopped.

Make Join.joinType final.

Verify in builder that there are no correlation variables where there
shouldn't be.

Refactor decorrelator.

Logged [CALCITE-1045] for remaining work.


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

Branch: refs/heads/branch-release
Commit: 505a9064b96a6c8399735fc2fa4d0ac9d5f3ed87
Parents: cd92b77
Author: Julian Hyde <jh...@apache.org>
Authored: Sat Jul 25 14:44:20 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:25 2016 -0800

----------------------------------------------------------------------
 .../adapter/enumerable/EnumerableJoin.java      |   42 +-
 .../adapter/enumerable/EnumerableJoinRule.java  |    9 +-
 .../adapter/enumerable/EnumerableMergeJoin.java |   22 +-
 .../enumerable/EnumerableMergeJoinRule.java     |    4 +-
 .../adapter/enumerable/EnumerableThetaJoin.java |   16 +-
 .../calcite/adapter/enumerable/RexImpTable.java |   14 +-
 .../apache/calcite/adapter/jdbc/JdbcRules.java  |   20 +-
 .../apache/calcite/interpreter/Bindables.java   |   17 +-
 .../org/apache/calcite/materialize/Lattice.java |    3 +-
 .../org/apache/calcite/plan/RelOptCluster.java  |    5 +-
 .../org/apache/calcite/plan/RelOptQuery.java    |    3 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  127 +-
 .../calcite/plan/SubstitutionVisitor.java       |   19 +-
 .../org/apache/calcite/plan/volcano/RelSet.java |   24 +-
 .../apache/calcite/plan/volcano/RelSubset.java  |   29 +-
 .../calcite/plan/volcano/VolcanoPlanner.java    |    2 +-
 .../org/apache/calcite/prepare/Prepare.java     |   10 +-
 .../org/apache/calcite/rel/AbstractRelNode.java |   16 +-
 .../java/org/apache/calcite/rel/RelNode.java    |   31 +-
 .../org/apache/calcite/rel/core/Correlate.java  |   15 +-
 .../apache/calcite/rel/core/CorrelationId.java  |   54 +-
 .../org/apache/calcite/rel/core/EquiJoin.java   |   15 +-
 .../java/org/apache/calcite/rel/core/Join.java  |   53 +-
 .../apache/calcite/rel/core/RelFactories.java   |   63 +-
 .../org/apache/calcite/rel/core/SemiJoin.java   |    4 +-
 .../apache/calcite/rel/externalize/RelJson.java |    2 +-
 .../apache/calcite/rel/logical/LogicalCalc.java |    5 +-
 .../calcite/rel/logical/LogicalCorrelate.java   |    3 +
 .../calcite/rel/logical/LogicalFilter.java      |   46 +-
 .../apache/calcite/rel/logical/LogicalJoin.java |   73 +-
 .../calcite/rel/metadata/RelMdUniqueKeys.java   |   11 +-
 .../org/apache/calcite/rel/rules/EquiJoin.java  |    3 +-
 .../calcite/rel/rules/JoinToCorrelateRule.java  |    5 +-
 .../rel/rules/JoinUnionTransposeRule.java       |    2 +-
 .../calcite/rel/rules/SubQueryRemoveRule.java   |  365 ++++
 .../apache/calcite/rel/stream/StreamRules.java  |   10 +-
 .../org/apache/calcite/rex/LogicVisitor.java    |  158 ++
 .../org/apache/calcite/rex/RexBiVisitor.java    |   52 +
 .../java/org/apache/calcite/rex/RexBuilder.java |    7 +-
 .../java/org/apache/calcite/rex/RexCall.java    |    8 +-
 .../apache/calcite/rex/RexCorrelVariable.java   |   12 +-
 .../org/apache/calcite/rex/RexDynamicParam.java |    4 +
 .../org/apache/calcite/rex/RexFieldAccess.java  |    4 +
 .../org/apache/calcite/rex/RexInputRef.java     |    4 +
 .../java/org/apache/calcite/rex/RexLiteral.java |   10 +-
 .../org/apache/calcite/rex/RexLocalRef.java     |    4 +
 .../java/org/apache/calcite/rex/RexNode.java    |    6 +
 .../java/org/apache/calcite/rex/RexOver.java    |    4 +
 .../java/org/apache/calcite/rex/RexProgram.java |    4 +
 .../org/apache/calcite/rex/RexRangeRef.java     |    4 +
 .../java/org/apache/calcite/rex/RexShuttle.java |   12 +-
 .../org/apache/calcite/rex/RexSubQuery.java     |  115 ++
 .../java/org/apache/calcite/rex/RexUtil.java    |  103 +-
 .../java/org/apache/calcite/rex/RexVisitor.java |    2 +
 .../org/apache/calcite/rex/RexVisitorImpl.java  |   12 +
 .../org/apache/calcite/schema/SchemaPlus.java   |    2 +-
 .../java/org/apache/calcite/sql/SqlKind.java    |    8 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  |    1 +
 .../sql2rel/DeduplicateCorrelateVariables.java  |   65 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java | 1646 ++++++++----------
 .../apache/calcite/sql2rel/RelFieldTrimmer.java |   12 +-
 .../sql2rel/RelStructuredTypeFlattener.java     |    7 +-
 .../calcite/sql2rel/SqlToRelConverter.java      |  390 +++--
 .../java/org/apache/calcite/tools/Programs.java |   54 +-
 .../org/apache/calcite/tools/RelBuilder.java    |   47 +-
 .../main/java/org/apache/calcite/util/Bug.java  |    5 +
 .../org/apache/calcite/test/CalciteAssert.java  |   10 +-
 .../apache/calcite/test/JdbcAdapterTest.java    |  122 +-
 .../java/org/apache/calcite/test/JdbcTest.java  |  214 ++-
 .../org/apache/calcite/test/LatticeTest.java    |   47 +-
 .../calcite/test/ReflectiveSchemaTest.java      |   21 +-
 .../org/apache/calcite/test/RelBuilderTest.java |   22 +
 .../apache/calcite/test/RelMetadataTest.java    |    3 +-
 .../apache/calcite/test/RelOptRulesTest.java    |  159 +-
 .../org/apache/calcite/test/RelOptTestBase.java |   30 +
 .../apache/calcite/test/RexTransformerTest.java |  210 +--
 .../calcite/test/SqlToRelConverterTest.java     |  421 ++++-
 .../apache/calcite/test/SqlToRelTestBase.java   |   36 +-
 .../apache/calcite/test/SqlValidatorTest.java   |   37 +
 .../enumerable/EnumerableCorrelateTest.java     |   29 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml |  706 ++++++++
 .../calcite/test/SqlToRelConverterTest.xml      |  582 ++++++-
 core/src/test/resources/sql/join.iq             |   18 +-
 core/src/test/resources/sql/misc.iq             |  121 +-
 core/src/test/resources/sql/subquery.iq         |   70 +-
 site/_docs/reference.md                         |   17 +-
 86 files changed, 5027 insertions(+), 1752 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
index 14fcf3b..0b86771 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
@@ -26,6 +26,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelNodes;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.EquiJoin;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -53,8 +54,8 @@ public class EnumerableJoin extends EquiJoin implements EnumerableRel {
       RexNode condition,
       ImmutableIntList leftKeys,
       ImmutableIntList rightKeys,
-      JoinRelType joinType,
-      Set<String> variablesStopped)
+      Set<CorrelationId> variablesSet,
+      JoinRelType joinType)
       throws InvalidRelException {
     super(
         cluster,
@@ -64,8 +65,17 @@ public class EnumerableJoin extends EquiJoin implements EnumerableRel {
         condition,
         leftKeys,
         rightKeys,
-        joinType,
-        variablesStopped);
+        variablesSet,
+        joinType);
+  }
+
+  @Deprecated // to be removed before 2.0
+  protected EnumerableJoin(RelOptCluster cluster, RelTraitSet traits,
+      RelNode left, RelNode right, RexNode condition, ImmutableIntList leftKeys,
+      ImmutableIntList rightKeys, JoinRelType joinType,
+      Set<String> variablesStopped) throws InvalidRelException {
+    this(cluster, traits, left, right, condition, leftKeys, rightKeys,
+        CorrelationId.setOf(variablesStopped), joinType);
   }
 
   /** Creates an EnumerableJoin. */
@@ -75,14 +85,28 @@ public class EnumerableJoin extends EquiJoin implements EnumerableRel {
       RexNode condition,
       ImmutableIntList leftKeys,
       ImmutableIntList rightKeys,
-      JoinRelType joinType,
-      Set<String> variablesStopped)
+      Set<CorrelationId> variablesSet,
+      JoinRelType joinType)
       throws InvalidRelException {
     final RelOptCluster cluster = left.getCluster();
     final RelTraitSet traitSet =
         cluster.traitSetOf(EnumerableConvention.INSTANCE);
     return new EnumerableJoin(cluster, traitSet, left, right, condition,
-        leftKeys, rightKeys, joinType, variablesStopped);
+        leftKeys, rightKeys, variablesSet, joinType);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public static EnumerableJoin create(
+      RelNode left,
+      RelNode right,
+      RexNode condition,
+      ImmutableIntList leftKeys,
+      ImmutableIntList rightKeys,
+      JoinRelType joinType,
+      Set<String> variablesStopped)
+      throws InvalidRelException {
+    return create(left, right, condition, leftKeys, rightKeys,
+        CorrelationId.setOf(variablesStopped), joinType);
   }
 
   @Override public EnumerableJoin copy(RelTraitSet traitSet, RexNode condition,
@@ -92,8 +116,8 @@ public class EnumerableJoin extends EquiJoin implements EnumerableRel {
     assert joinInfo.isEqui();
     try {
       return new EnumerableJoin(getCluster(), traitSet, left, right,
-          condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType,
-          variablesStopped);
+          condition, joinInfo.leftKeys, joinInfo.rightKeys, variablesSet,
+          joinType);
     } catch (InvalidRelException e) {
       // Semantic error not possible. Must be a bug. Convert to
       // internal error.

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
index 6ffc912..88655ba 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
@@ -43,7 +43,7 @@ class EnumerableJoinRule extends ConverterRule {
 
   @Override public RelNode convert(RelNode rel) {
     LogicalJoin join = (LogicalJoin) rel;
-    List<RelNode> newInputs = new ArrayList<RelNode>();
+    List<RelNode> newInputs = new ArrayList<>();
     for (RelNode input : join.getInputs()) {
       if (!(input.getConvention() instanceof EnumerableConvention)) {
         input =
@@ -65,8 +65,7 @@ class EnumerableJoinRule extends ConverterRule {
       // if it is an inner join.
       try {
         return new EnumerableThetaJoin(cluster, traitSet, left, right,
-            join.getCondition(), join.getJoinType(),
-            join.getVariablesStopped());
+            join.getCondition(), join.getVariablesSet(), join.getJoinType());
       } catch (InvalidRelException e) {
         EnumerableRules.LOGGER.fine(e.toString());
         return null;
@@ -82,8 +81,8 @@ class EnumerableJoinRule extends ConverterRule {
           info.getEquiCondition(left, right, cluster.getRexBuilder()),
           info.leftKeys,
           info.rightKeys,
-          join.getJoinType(),
-          join.getVariablesStopped());
+          join.getVariablesSet(),
+          join.getJoinType());
     } catch (InvalidRelException e) {
       EnumerableRules.LOGGER.fine(e.toString());
       return null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java
index be0d821..18419e3 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java
@@ -28,6 +28,7 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.EquiJoin;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -56,16 +57,25 @@ public class EnumerableMergeJoin extends EquiJoin implements EnumerableRel {
       RexNode condition,
       ImmutableIntList leftKeys,
       ImmutableIntList rightKeys,
-      JoinRelType joinType,
-      Set<String> variablesStopped)
+      Set<CorrelationId> variablesSet,
+      JoinRelType joinType)
       throws InvalidRelException {
     super(cluster, traits, left, right, condition, leftKeys, rightKeys,
-        joinType, variablesStopped);
+        variablesSet, joinType);
     final List<RelCollation> collations =
         traits.getTraits(RelCollationTraitDef.INSTANCE);
     assert collations == null || RelCollations.contains(collations, leftKeys);
   }
 
+  @Deprecated // to be removed before 2.0
+  EnumerableMergeJoin(RelOptCluster cluster, RelTraitSet traits, RelNode left,
+      RelNode right, RexNode condition, ImmutableIntList leftKeys,
+      ImmutableIntList rightKeys, JoinRelType joinType,
+      Set<String> variablesStopped) throws InvalidRelException {
+    this(cluster, traits, left, right, condition, leftKeys, rightKeys,
+        CorrelationId.setOf(variablesStopped), joinType);
+  }
+
   public static EnumerableMergeJoin create(RelNode left, RelNode right,
       RexLiteral condition, ImmutableIntList leftKeys,
       ImmutableIntList rightKeys, JoinRelType joinType)
@@ -78,7 +88,7 @@ public class EnumerableMergeJoin extends EquiJoin implements EnumerableRel {
       traitSet = traitSet.replace(collations);
     }
     return new EnumerableMergeJoin(cluster, traitSet, left, right, condition,
-        leftKeys, rightKeys, joinType, ImmutableSet.<String>of());
+        leftKeys, rightKeys, ImmutableSet.<CorrelationId>of(), joinType);
   }
 
   @Override public EnumerableMergeJoin copy(RelTraitSet traitSet,
@@ -88,8 +98,8 @@ public class EnumerableMergeJoin extends EquiJoin implements EnumerableRel {
     assert joinInfo.isEqui();
     try {
       return new EnumerableMergeJoin(getCluster(), traitSet, left, right,
-          condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType,
-          variablesStopped);
+          condition, joinInfo.leftKeys, joinInfo.rightKeys, variablesSet,
+          joinType);
     } catch (InvalidRelException e) {
       // Semantic error not possible. Must be a bug. Convert to
       // internal error.

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java
index 51f09f4..9dd0ce1 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java
@@ -99,8 +99,8 @@ class EnumerableMergeJoinRule extends ConverterRule {
           info.getEquiCondition(left, right, cluster.getRexBuilder()),
           info.leftKeys,
           info.rightKeys,
-          join.getJoinType(),
-          join.getVariablesStopped());
+          join.getVariablesSet(),
+          join.getJoinType());
     } catch (InvalidRelException e) {
       EnumerableRules.LOGGER.fine(e.toString());
       return null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java
index bf4516a..e28ddfc 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java
@@ -27,6 +27,7 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
@@ -43,11 +44,20 @@ import java.util.Set;
  * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}
  * that allows conditions that are not just {@code =} (equals). */
 public class EnumerableThetaJoin extends Join implements EnumerableRel {
+  /** Creates an EnumerableThetaJoin. */
+  protected EnumerableThetaJoin(RelOptCluster cluster, RelTraitSet traits,
+      RelNode left, RelNode right, RexNode condition,
+      Set<CorrelationId> variablesSet, JoinRelType joinType)
+      throws InvalidRelException {
+    super(cluster, traits, left, right, condition, variablesSet, joinType);
+  }
+
+  @Deprecated // to be removed before 2.0
   protected EnumerableThetaJoin(RelOptCluster cluster, RelTraitSet traits,
       RelNode left, RelNode right, RexNode condition, JoinRelType joinType,
       Set<String> variablesStopped) throws InvalidRelException {
-    super(cluster, traits, left, right, condition, joinType,
-        variablesStopped);
+    this(cluster, traits, left, right, condition,
+        CorrelationId.setOf(variablesStopped), joinType);
   }
 
   @Override public EnumerableThetaJoin copy(RelTraitSet traitSet,
@@ -55,7 +65,7 @@ public class EnumerableThetaJoin extends Join implements EnumerableRel {
       boolean semiJoinDone) {
     try {
       return new EnumerableThetaJoin(getCluster(), traitSet, left, right,
-          condition, joinType, variablesStopped);
+          condition, variablesSet, joinType);
     } catch (InvalidRelException e) {
       // Semantic error not possible. Must be a bug. Convert to
       // internal error.

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 511584b..2633490 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
@@ -359,10 +359,8 @@ public class RexImpTable {
       public T get() {
         try {
           return constructor.newInstance();
-        } catch (InstantiationException e) {
-          throw new IllegalStateException(
-              "Unable to instantiate aggregate implementor " + constructor, e);
-        } catch (IllegalAccessException | InvocationTargetException e) {
+        } catch (InstantiationException | IllegalAccessException
+            | InvocationTargetException e) {
           throw new IllegalStateException(
               "Error while creating aggregate implementor " + constructor, e);
         }
@@ -1883,11 +1881,9 @@ public class RexImpTable {
         return translator.translate(operands.get(0),
             negate ? NullAs.IS_NOT_NULL : NullAs.IS_NULL);
       } else {
-        return maybeNegate(
-            negate == seek,
-            translator.translate(
-                operands.get(0),
-                negate == seek ? NullAs.TRUE : NullAs.FALSE));
+        return maybeNegate(negate == seek,
+            translator.translate(operands.get(0),
+                seek ? NullAs.FALSE : NullAs.TRUE));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index 5fbb08f..ca80d29 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -35,6 +35,7 @@ import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Intersect;
 import org.apache.calcite.rel.core.Join;
@@ -159,8 +160,8 @@ public class JdbcRules {
             newInputs.get(0),
             newInputs.get(1),
             join.getCondition(),
-            join.getJoinType(),
-            join.getVariablesStopped());
+            join.getVariablesSet(),
+            join.getJoinType());
       } catch (InvalidRelException e) {
         LOGGER.fine(e.toString());
         return null;
@@ -211,6 +212,15 @@ public class JdbcRules {
 
   /** Join operator implemented in JDBC convention. */
   public static class JdbcJoin extends Join implements JdbcRel {
+    /** Creates a JdbcJoin. */
+    protected JdbcJoin(RelOptCluster cluster, RelTraitSet traitSet,
+        RelNode left, RelNode right, RexNode condition,
+        Set<CorrelationId> variablesSet, JoinRelType joinType)
+        throws InvalidRelException {
+      super(cluster, traitSet, left, right, condition, variablesSet, joinType);
+    }
+
+    @Deprecated // to be removed before 2.0
     protected JdbcJoin(
         RelOptCluster cluster,
         RelTraitSet traitSet,
@@ -220,8 +230,8 @@ public class JdbcRules {
         JoinRelType joinType,
         Set<String> variablesStopped)
         throws InvalidRelException {
-      super(cluster, traitSet, left, right, condition,
-          joinType, variablesStopped);
+      this(cluster, traitSet, left, right, condition,
+          CorrelationId.setOf(variablesStopped), joinType);
     }
 
     @Override public JdbcJoin copy(RelTraitSet traitSet, RexNode condition,
@@ -229,7 +239,7 @@ public class JdbcRules {
         boolean semiJoinDone) {
       try {
         return new JdbcJoin(getCluster(), traitSet, left, right,
-            condition, joinType, variablesStopped);
+            condition, variablesSet, joinType);
       } catch (InvalidRelException e) {
         // Semantic error not possible. Must be a bug. Convert to
         // internal error.

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
index a4474b4..9d4b250 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
@@ -37,6 +37,7 @@ import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -417,25 +418,33 @@ public class Bindables {
           convert(join.getRight(),
               join.getRight().getTraitSet()
                   .replace(BindableConvention.INSTANCE)),
-          join.getCondition(), join.getJoinType(), join.getVariablesStopped());
+          join.getCondition(), join.getVariablesSet(), join.getJoinType());
     }
   }
 
   /** Implementation of {@link org.apache.calcite.rel.core.Join} in
    * bindable calling convention. */
   public static class BindableJoin extends Join implements BindableRel {
+    /** Creates a BindableJoin. */
+    protected BindableJoin(RelOptCluster cluster, RelTraitSet traitSet,
+        RelNode left, RelNode right, RexNode condition,
+        Set<CorrelationId> variablesSet, JoinRelType joinType) {
+      super(cluster, traitSet, left, right, condition, variablesSet, joinType);
+    }
+
+    @Deprecated // to be removed before 2.0
     protected BindableJoin(RelOptCluster cluster, RelTraitSet traitSet,
         RelNode left, RelNode right, RexNode condition, JoinRelType joinType,
         Set<String> variablesStopped) {
-      super(cluster, traitSet, left, right, condition, joinType,
-          variablesStopped);
+      this(cluster, traitSet, left, right, condition,
+          CorrelationId.setOf(variablesStopped), joinType);
     }
 
     public BindableJoin copy(RelTraitSet traitSet, RexNode conditionExpr,
         RelNode left, RelNode right, JoinRelType joinType,
         boolean semiJoinDone) {
       return new BindableJoin(getCluster(), traitSet, left, right,
-          conditionExpr, joinType, variablesStopped);
+          conditionExpr, variablesSet, joinType);
     }
 
     public Class<Object[]> getElementType() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/materialize/Lattice.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/Lattice.java b/core/src/main/java/org/apache/calcite/materialize/Lattice.java
index 62ab07f..244a245 100644
--- a/core/src/main/java/org/apache/calcite/materialize/Lattice.java
+++ b/core/src/main/java/org/apache/calcite/materialize/Lattice.java
@@ -44,7 +44,6 @@ import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
-import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.graph.DefaultDirectedGraph;
 import org.apache.calcite.util.graph.DefaultEdge;
@@ -260,7 +259,7 @@ public class Lattice {
     final StringBuilder groupBuf = new StringBuilder("\nGROUP BY ");
     int k = 0;
     final Set<String> columnNames = Sets.newHashSet();
-    for (int i : BitSets.toIter(groupSet)) {
+    for (int i : groupSet) {
       if (k++ > 0) {
         buf.append(", ");
         groupBuf.append(", ");

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java b/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
index c4592db..241f4e0 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.plan;
 
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.MetadataFactory;
 import org.apache.calcite.rel.metadata.MetadataFactoryImpl;
@@ -143,8 +144,8 @@ public class RelOptCluster {
    * Constructs a new id for a correlating variable. It is unique within the
    * whole query.
    */
-  public int createCorrel() {
-    return nextCorrel.getAndIncrement();
+  public CorrelationId createCorrel() {
+    return new CorrelationId(nextCorrel.getAndIncrement());
   }
 
   /** Returns the default trait set for this cluster. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java b/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java
index 2c2703d..34f5c5f 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptQuery.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.plan;
 
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
 
@@ -35,7 +36,7 @@ public class RelOptQuery {
   /**
    * Prefix to the name of correlating variables.
    */
-  public static final String CORREL_PREFIX = "$cor";
+  public static final String CORREL_PREFIX = CorrelationId.CORREL_PREFIX;
 
   //~ Instance fields --------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 b932cef..eeaea21 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -26,6 +26,7 @@ import org.apache.calcite.rel.RelVisitor;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -60,6 +61,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlExplainLevel;
@@ -83,7 +85,9 @@ import org.apache.calcite.util.mapping.Mappings;
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -155,7 +159,7 @@ public abstract class RelOptUtil {
    * Returns a list of variables set by a relational expression or its
    * descendants.
    */
-  public static Set<String> getVariablesSet(RelNode rel) {
+  public static Set<CorrelationId> getVariablesSet(RelNode rel) {
     VariableSetVisitor visitor = new VariableSetVisitor();
     go(visitor, rel);
     return visitor.variables;
@@ -165,19 +169,18 @@ public abstract class RelOptUtil {
    * Returns a set of distinct variables set by <code>rel0</code> and used by
    * <code>rel1</code>.
    */
-  public static List<String> getVariablesSetAndUsed(
-      RelNode rel0,
+  public static List<CorrelationId> getVariablesSetAndUsed(RelNode rel0,
       RelNode rel1) {
-    Set<String> set = getVariablesSet(rel0);
+    Set<CorrelationId> set = getVariablesSet(rel0);
     if (set.size() == 0) {
       return ImmutableList.of();
     }
-    Set<String> used = getVariablesUsed(rel1);
+    Set<CorrelationId> used = getVariablesUsed(rel1);
     if (used.size() == 0) {
       return ImmutableList.of();
     }
-    final List<String> result = new ArrayList<>();
-    for (String s : set) {
+    final List<CorrelationId> result = new ArrayList<>();
+    for (CorrelationId s : set) {
       if (used.contains(s) && !result.contains(s)) {
         result.add(s);
       }
@@ -187,24 +190,45 @@ public abstract class RelOptUtil {
 
   /**
    * Returns a set of variables used by a relational expression or its
-   * descendants. The set may contain duplicates. The item type is the same as
-   * {@link org.apache.calcite.rex.RexVariable#getName}
-   */
-  public static Set<String> getVariablesUsed(RelNode rel) {
-    final VariableUsedVisitor vuv = new VariableUsedVisitor();
-    RelShuttle visitor = new RelHomogeneousShuttle() {
-      @Override public RelNode visit(RelNode other) {
-        other.collectVariablesUsed(vuv.variables);
-        other.accept(vuv);
-        RelNode result = super.visit(other);
-        // Important! Remove stopped variables AFTER we visit
-        // children. (which what super.visit() does)
-        vuv.variables.removeAll(other.getVariablesStopped());
-        return result;
-      }
-    };
+   * descendants.
+   *
+   * <p>The set may contain "duplicates" (variables with different ids that,
+   * when resolved, will reference the same source relational expression).
+   *
+   * <p>The item type is the same as
+   * {@link org.apache.calcite.rex.RexCorrelVariable#id}.
+   */
+  public static Set<CorrelationId> getVariablesUsed(RelNode rel) {
+    CorrelationCollector visitor = new CorrelationCollector();
     rel.accept(visitor);
-    return vuv.variables;
+    return visitor.vuv.variables;
+  }
+
+  /** Finds which columns of a correlation variable are used within a
+   * relational expression. */
+  public static ImmutableBitSet correlationColumns(CorrelationId id,
+      RelNode rel) {
+    final CorrelationCollector collector = new CorrelationCollector();
+    rel.accept(collector);
+    final ImmutableBitSet.Builder builder = ImmutableBitSet.builder();
+    for (int field : collector.vuv.variableFields.get(id)) {
+      if (field >= 0) {
+        builder.set(field);
+      }
+    }
+    return builder.build();
+  }
+
+  /** Returns true, and calls {@link Litmus#succeed()} if a given relational
+   * expression does not contain a given correlation. */
+  public static boolean notContainsCorrelation(RelNode r,
+      CorrelationId correlationId, Litmus litmus) {
+    final Set<CorrelationId> set = getVariablesUsed(r);
+    if (!set.contains(correlationId)) {
+      return litmus.succeed();
+    } else {
+      return litmus.fail("contains " + correlationId);
+    }
   }
 
   /**
@@ -2946,6 +2970,7 @@ public abstract class RelOptUtil {
     public Logic negate() {
       switch (this) {
       case UNKNOWN_AS_FALSE:
+      case TRUE:
         return UNKNOWN_AS_TRUE;
       case UNKNOWN_AS_TRUE:
         return UNKNOWN_AS_FALSE;
@@ -2982,7 +3007,8 @@ public abstract class RelOptUtil {
     // Pushing sub-queries is OK in principle (if they don't reference both
     // sides of the join via correlating variables) but we'd rather not do it
     // yet.
-    if (!containsGet(joinCond)) {
+    if (!containsGet(joinCond)
+        && RexUtil.SubQueryFinder.find(joinCond) == null) {
       joinCond = pushDownEqualJoinConditions(
           joinCond, leftCount, rightCount, extraLeftExprs, extraRightExprs);
     }
@@ -3177,7 +3203,7 @@ public abstract class RelOptUtil {
 
   /** Visitor that finds all variables used but not stopped in an expression. */
   private static class VariableSetVisitor extends RelVisitor {
-    final Set<String> variables = new HashSet<>();
+    final Set<CorrelationId> variables = new HashSet<>();
 
     // implement RelVisitor
     public void visit(
@@ -3189,18 +3215,42 @@ public abstract class RelOptUtil {
 
       // Important! Remove stopped variables AFTER we visit children
       // (which what super.visit() does)
-      variables.removeAll(p.getVariablesStopped());
+      variables.removeAll(p.getVariablesSet());
     }
   }
 
   /** Visitor that finds all variables used in an expression. */
   public static class VariableUsedVisitor extends RexShuttle {
-    public final Set<String> variables = new LinkedHashSet<>();
+    public final Set<CorrelationId> variables = new LinkedHashSet<>();
+    public final Multimap<CorrelationId, Integer> variableFields =
+        LinkedHashMultimap.create();
+    private final RelShuttle relShuttle;
+
+    public VariableUsedVisitor(RelShuttle relShuttle) {
+      this.relShuttle = relShuttle;
+    }
 
-    public RexNode visitCorrelVariable(RexCorrelVariable p) {
-      variables.add(p.getName());
+    @Override public RexNode visitCorrelVariable(RexCorrelVariable p) {
+      variables.add(p.id);
+      variableFields.put(p.id, -1);
       return p;
     }
+
+    @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+      if (fieldAccess.getReferenceExpr() instanceof RexCorrelVariable) {
+        final RexCorrelVariable v =
+            (RexCorrelVariable) fieldAccess.getReferenceExpr();
+        variableFields.put(v.id, fieldAccess.getField().getIndex());
+      }
+      return super.visitFieldAccess(fieldAccess);
+    }
+
+    @Override public RexNode visitSubQuery(RexSubQuery subQuery) {
+      if (relShuttle != null) {
+        subQuery.rel.accept(relShuttle); // look inside sub-queries
+      }
+      return super.visitSubQuery(subQuery);
+    }
   }
 
   /** Shuttle that finds the set of inputs that are used. */
@@ -3462,6 +3512,23 @@ public abstract class RelOptUtil {
       return BOTH;
     }
   }
+
+  /** Shuttle that finds correlation variables inside a given relational
+   * expression, including those that are inside
+   * {@link RexSubQuery sub-queries}. */
+  private static class CorrelationCollector extends RelHomogeneousShuttle {
+    private final VariableUsedVisitor vuv = new VariableUsedVisitor(this);
+
+    @Override public RelNode visit(RelNode other) {
+      other.collectVariablesUsed(vuv.variables);
+      other.accept(vuv);
+      RelNode result = super.visit(other);
+      // Important! Remove stopped variables AFTER we visit
+      // children. (which what super.visit() does)
+      vuv.variables.removeAll(other.getVariablesSet());
+      return result;
+    }
+  }
 }
 
 // End RelOptUtil.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 8dd0d01..ccbe2c2 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -24,6 +24,7 @@ import org.apache.calcite.rel.RelNode;
 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.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -249,7 +250,7 @@ public class SubstitutionVisitor {
       final MutableRel left = toMutable(join.getLeft());
       final MutableRel right = toMutable(join.getRight());
       return MutableJoin.of(join.getCluster(), left, right,
-          join.getCondition(), join.getJoinType(), join.getVariablesStopped());
+          join.getCondition(), join.getJoinType(), join.getVariablesSet());
     }
     if (rel instanceof Sort) {
       final Sort sort = (Sort) rel;
@@ -646,7 +647,7 @@ public class SubstitutionVisitor {
     case JOIN:
       final MutableJoin join = (MutableJoin) node;
       return LogicalJoin.create(fromMutable(join.getLeft()), fromMutable(join.getRight()),
-          join.getCondition(), join.getJoinType(), join.getVariablesStopped());
+          join.getCondition(), join.getVariablesSet(), join.getJoinType());
     default:
       throw new AssertionError(node.deep());
     }
@@ -690,7 +691,7 @@ public class SubstitutionVisitor {
       final MutableJoin join = (MutableJoin) node;
       return MutableJoin.of(join.cluster, copyMutable(join.getLeft()),
           copyMutable(join.getRight()), join.getCondition(), join.getJoinType(),
-          join.getVariablesStopped());
+          join.getVariablesSet());
     default:
       throw new AssertionError(node.deep());
     }
@@ -1980,7 +1981,7 @@ public class SubstitutionVisitor {
     //~ Instance fields --------------------------------------------------------
 
     protected final RexNode condition;
-    protected final ImmutableSet<String> variablesStopped;
+    protected final ImmutableSet<CorrelationId> variablesSet;
 
     /**
      * Values must be of enumeration {@link JoinRelType}, except that
@@ -1994,10 +1995,10 @@ public class SubstitutionVisitor {
         MutableRel right,
         RexNode condition,
         JoinRelType joinType,
-        Set<String> variablesStopped) {
+        Set<CorrelationId> variablesSet) {
       super(MutableRelType.JOIN, left.cluster, rowType, left, right);
       this.condition = Preconditions.checkNotNull(condition);
-      this.variablesStopped = ImmutableSet.copyOf(variablesStopped);
+      this.variablesSet = ImmutableSet.copyOf(variablesSet);
       this.joinType = Preconditions.checkNotNull(joinType);
     }
 
@@ -2009,13 +2010,13 @@ public class SubstitutionVisitor {
       return joinType;
     }
 
-    public ImmutableSet getVariablesStopped() {
-      return variablesStopped;
+    public ImmutableSet<CorrelationId> getVariablesSet() {
+      return variablesSet;
     }
 
     static MutableJoin of(RelOptCluster cluster, MutableRel left,
         MutableRel right, RexNode condition, JoinRelType joinType,
-        Set<String> variablesStopped) {
+        Set<CorrelationId> variablesStopped) {
       List<RelDataTypeField> fieldList = Collections.emptyList();
       RelDataType rowType =
           Join.deriveJoinRowType(left.getRowType(), right.getRowType(),

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
index 93d8509..cf78dff 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
@@ -22,6 +22,7 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
@@ -47,21 +48,20 @@ class RelSet {
 
   //~ Instance fields --------------------------------------------------------
 
-  final List<RelNode> rels = new ArrayList<RelNode>();
+  final List<RelNode> rels = new ArrayList<>();
   /**
    * Relational expressions that have a subset in this set as a child. This
    * is a multi-set. If multiple relational expressions in this set have the
    * same parent, there will be multiple entries.
    */
-  final List<RelNode> parents = new ArrayList<RelNode>();
-  final List<RelSubset> subsets = new ArrayList<RelSubset>();
+  final List<RelNode> parents = new ArrayList<>();
+  final List<RelSubset> subsets = new ArrayList<>();
 
   /**
    * List of {@link AbstractConverter} objects which have not yet been
    * satisfied.
    */
-  final List<AbstractConverter> abstractConverters =
-      new ArrayList<AbstractConverter>();
+  final List<AbstractConverter> abstractConverters = new ArrayList<>();
 
   /**
    * Set to the superseding set when this is found to be equivalent to another
@@ -71,15 +71,15 @@ class RelSet {
   RelNode rel;
 
   /**
-   * Names of variables which are set by relational expressions in this set
+   * Variables that are set by relational expressions in this set
    * and available for use by parent and child expressions.
    */
-  final Set<String> variablesPropagated;
+  final Set<CorrelationId> variablesPropagated;
 
   /**
-   * Names of variables which are used by relational expressions in this set.
+   * Variables that are used by relational expressions in this set.
    */
-  final Set<String> variablesUsed;
+  final Set<CorrelationId> variablesUsed;
   final int id;
 
   /**
@@ -91,8 +91,8 @@ class RelSet {
 
   RelSet(
       int id,
-      Set<String> variablesPropagated,
-      Set<String> variablesUsed) {
+      Set<CorrelationId> variablesPropagated,
+      Set<CorrelationId> variablesUsed) {
     this.id = id;
     this.variablesPropagated = variablesPropagated;
     this.variablesUsed = variablesUsed;
@@ -275,7 +275,7 @@ class RelSet {
     }
 
     // Make sure the cost changes as a result of merging are propagated.
-    Set<RelSubset> activeSet = new HashSet<RelSubset>();
+    Set<RelSubset> activeSet = new HashSet<>();
     for (RelNode parentRel : getParentRels()) {
       final RelSubset parentSubset = planner.getSubset(parentRel);
       parentSubset.propagateCostImprovements(

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
index bb42218..3fede02 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
@@ -28,6 +28,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.AbstractRelNode;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.util.Litmus;
@@ -136,14 +137,6 @@ public class RelSubset extends AbstractRelNode {
     }
   }
 
-  public Set<String> getVariablesSet() {
-    return set.variablesPropagated;
-  }
-
-  public Set<String> getVariablesUsed() {
-    return set.variablesUsed;
-  }
-
   public RelNode getBest() {
     return best;
   }
@@ -164,7 +157,7 @@ public class RelSubset extends AbstractRelNode {
     }
   }
 
-  public void explain(RelWriter pw) {
+  @Override public void explain(RelWriter pw) {
     // Not a typical implementation of "explain". We don't gather terms &
     // values to be printed later. We actually do the work.
     String s = getDescription();
@@ -178,7 +171,7 @@ public class RelSubset extends AbstractRelNode {
     pw.done(input);
   }
 
-  protected String computeDigest() {
+  @Override protected String computeDigest() {
     StringBuilder digest = new StringBuilder("Subset#");
     digest.append(set.id);
     for (RelTrait trait : traitSet) {
@@ -276,13 +269,13 @@ public class RelSubset extends AbstractRelNode {
           "rowtype of set", getRowType(), Litmus.THROW);
     }
     set.addInternal(rel);
-    Set<String> variablesSet = RelOptUtil.getVariablesSet(rel);
-    Set<String> variablesStopped = rel.getVariablesStopped();
+    Set<CorrelationId> variablesSet = RelOptUtil.getVariablesSet(rel);
+    Set<CorrelationId> variablesStopped = rel.getVariablesSet();
     if (false) {
-      Set<String> variablesPropagated =
+      Set<CorrelationId> variablesPropagated =
           Util.minus(variablesSet, variablesStopped);
       assert set.variablesPropagated.containsAll(variablesPropagated);
-      Set<String> variablesUsed = RelOptUtil.getVariablesUsed(rel);
+      Set<CorrelationId> variablesUsed = RelOptUtil.getVariablesUsed(rel);
       assert set.variablesUsed.containsAll(variablesUsed);
     }
   }
@@ -375,12 +368,12 @@ public class RelSubset extends AbstractRelNode {
     }
   }
 
-  public void collectVariablesUsed(Set<String> variableSet) {
-    variableSet.addAll(getVariablesUsed());
+  @Override public void collectVariablesUsed(Set<CorrelationId> variableSet) {
+    variableSet.addAll(set.variablesUsed);
   }
 
-  public void collectVariablesSet(Set<String> variableSet) {
-    variableSet.addAll(getVariablesSet());
+  @Override public void collectVariablesSet(Set<CorrelationId> variableSet) {
+    variableSet.addAll(set.variablesPropagated);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
index 4186232..425ebec 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
@@ -1759,7 +1759,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
           nextSetId++,
           Util.minus(
               RelOptUtil.getVariablesSet(rel),
-              rel.getVariablesStopped()),
+              rel.getVariablesSet()),
           RelOptUtil.getVariablesUsed(rel));
       this.allSets.add(set);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 3530e93..e624d1d 100644
--- a/core/src/main/java/org/apache/calcite/prepare/Prepare.java
+++ b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
@@ -87,9 +87,13 @@ public abstract class Prepare {
   public static final TryThreadLocal<Boolean> THREAD_TRIM =
       TryThreadLocal.of(false);
 
-  /** Temporary, while CALCITE-816 is under development.
+  /** Temporary, until
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1045">[CALCITE-1045]
+   * Decorrelate sub-queries in Project and Join</a> is fixed.
    *
-   * @see org.apache.calcite.util.Util#deprecated(Object, boolean) */
+   * <p>The default is false, meaning do not expand queries during sql-to-rel,
+   * but a few tests override and set it to true. After CALCITE-1045
+   * is fixed, remove those overrides and use false everywhere. */
   public static final TryThreadLocal<Boolean> THREAD_EXPAND =
       TryThreadLocal.of(false);
 
@@ -209,6 +213,7 @@ public abstract class Prepare {
 
     SqlToRelConverter sqlToRelConverter =
         getSqlToRelConverter(validator, catalogReader);
+    sqlToRelConverter.setExpand(THREAD_EXPAND.get());
 
     SqlExplain sqlExplain = null;
     if (sqlQuery.getKind() == SqlKind.EXPLAIN) {
@@ -344,6 +349,7 @@ public abstract class Prepare {
         getSqlToRelConverter(
             getSqlValidator(), catalogReader);
     converter.setTrimUnusedFields(shouldTrim(root.rel));
+    converter.setExpand(THREAD_EXPAND.get());
     final boolean ordered = !root.collation.getFieldCollations().isEmpty();
     final boolean dml = SqlKind.DML.contains(root.kind);
     return root.withRel(converter.trimUnusedFields(dml || ordered, root.rel));

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
index 1938447..75a546c 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -26,6 +26,7 @@ import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.externalize.RelWriterImpl;
 import org.apache.calcite.rel.metadata.Metadata;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
@@ -40,6 +41,7 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -237,15 +239,19 @@ public abstract class AbstractRelNode implements RelNode {
     return 1.0;
   }
 
-  public Set<String> getVariablesStopped() {
-    return Collections.emptySet();
+  public final Set<String> getVariablesStopped() {
+    return CorrelationId.names(getVariablesSet());
   }
 
-  public void collectVariablesUsed(Set<String> variableSet) {
+  public Set<CorrelationId> getVariablesSet() {
+    return ImmutableSet.of();
+  }
+
+  public void collectVariablesUsed(Set<CorrelationId> variableSet) {
     // for default case, nothing to do
   }
 
-  public void collectVariablesSet(Set<String> variableSet) {
+  public void collectVariablesSet(Set<CorrelationId> variableSet) {
   }
 
   public void childrenAccept(RelVisitor visitor) {
@@ -305,7 +311,7 @@ public abstract class AbstractRelNode implements RelNode {
 
   public RelNode onRegister(RelOptPlanner planner) {
     List<RelNode> oldInputs = getInputs();
-    List<RelNode> inputs = new ArrayList<RelNode>(oldInputs.size());
+    List<RelNode> inputs = new ArrayList<>(oldInputs.size());
     for (final RelNode input : oldInputs) {
       RelNode e = planner.ensureRegistered(input, null);
       if (e != input) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/RelNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelNode.java b/core/src/main/java/org/apache/calcite/rel/RelNode.java
index 29dc023..973bc98 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelNode.java
@@ -23,6 +23,7 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptQuery;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.metadata.Metadata;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
@@ -175,18 +176,40 @@ public interface RelNode extends RelOptNode, Cloneable {
   double getRows();
 
   /**
-   * Returns the names of variables which are set in this relational
+   * Returns the names of variables that are set in this relational
    * expression but also used and therefore not available to parents of this
    * relational expression.
+   *
    * <p>Note: only {@link org.apache.calcite.rel.core.Correlate} should set
-   * variables</p>
+   * variables.
+   *
+   * <p>Note: {@link #getVariablesSet()} is equivalent but returns
+   * {@link CorrelationId} rather than their names. It is preferable except for
+   * calling old methods that require a set of strings.
    *
    * @return Names of variables which are set in this relational
    *   expression
+   *
+   * @deprecated Use {@link #getVariablesSet()}
+   * and {@link CorrelationId#names(Set)}
    */
+  @Deprecated // to be removed before 2.0
   Set<String> getVariablesStopped();
 
   /**
+   * Returns the variables that are set in this relational
+   * expression but also used and therefore not available to parents of this
+   * relational expression.
+   *
+   * <p>Note: only {@link org.apache.calcite.rel.core.Correlate} should set
+   * variables.
+   *
+   * @return Names of variables which are set in this relational
+   *   expression
+   */
+  Set<CorrelationId> getVariablesSet();
+
+  /**
    * Collects variables known to be used by this expression or its
    * descendants. By default, no such information is available and must be
    * derived by analyzing sub-expressions, but some optimizer implementations
@@ -194,7 +217,7 @@ public interface RelNode extends RelOptNode, Cloneable {
    *
    * @param variableSet receives variables used
    */
-  void collectVariablesUsed(Set<String> variableSet);
+  void collectVariablesUsed(Set<CorrelationId> variableSet);
 
   /**
    * Collects variables set by this expression.
@@ -202,7 +225,7 @@ public interface RelNode extends RelOptNode, Cloneable {
    *
    * @param variableSet receives variables known to be set by
    */
-  void collectVariablesSet(Set<String> variableSet);
+  void collectVariablesSet(Set<CorrelationId> variableSet);
 
   /**
    * Interacts with the {@link RelVisitor} in a

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Correlate.java b/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
index b1c0e04..982a762 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
@@ -19,6 +19,7 @@ package org.apache.calcite.rel.core;
 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.BiRel;
 import org.apache.calcite.rel.RelInput;
@@ -29,6 +30,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 
 import com.google.common.collect.ImmutableSet;
 
@@ -109,6 +111,11 @@ public abstract class Correlate extends BiRel {
 
   //~ Methods ----------------------------------------------------------------
 
+  @Override public boolean isValid(Litmus litmus) {
+    return super.isValid(litmus)
+        && RelOptUtil.notContainsCorrelation(left, correlationId, litmus);
+  }
+
   @Override public Correlate copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert inputs.size() == 2;
     return copy(traitSet,
@@ -133,8 +140,8 @@ public abstract class Correlate extends BiRel {
     case INNER:
       // LogicalJoin is used to share the code of column names deduplication
       final LogicalJoin join = LogicalJoin.create(left, right,
-          getCluster().getRexBuilder().makeLiteral(true), joinType.toJoinType(),
-          ImmutableSet.<String>of());
+          getCluster().getRexBuilder().makeLiteral(true),
+          ImmutableSet.<CorrelationId>of(), joinType.toJoinType());
       return join.deriveRowType();
     case ANTI:
     case SEMI:
@@ -174,8 +181,8 @@ public abstract class Correlate extends BiRel {
     return requiredColumns;
   }
 
-  @Override public Set<String> getVariablesStopped() {
-    return ImmutableSet.of(correlationId.getName());
+  @Override public Set<CorrelationId> getVariablesSet() {
+    return ImmutableSet.of(correlationId);
   }
 
   @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java b/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java
index 2b2558c..d5d74c9 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java
@@ -16,39 +16,51 @@
  */
 package org.apache.calcite.rel.core;
 
+import com.google.common.collect.ImmutableSet;
+
+import java.util.Set;
+
 /**
  * Describes the necessary parameters for an implementation in order to
  * identify and set dynamic variables
  */
 public class CorrelationId implements Cloneable, Comparable<CorrelationId> {
-  private static final String CORREL_PREFIX = "$cor";
+  /**
+   * Prefix to the name of correlating variables.
+   */
+  public static final String CORREL_PREFIX = "$cor";
 
   private final int id;
   private final String name;
 
   /**
    * Creates a correlation identifier.
+   */
+  private CorrelationId(int id, String name) {
+    this.id = id;
+    this.name = name;
+  }
+
+  /**
+   * Creates a correlation identifier.
    * This is a type-safe wrapper over int.
    *
    * @param id     Identifier
    */
   public CorrelationId(int id) {
-    this.id = id;
-    this.name = CORREL_PREFIX + id;
+    this(id, CORREL_PREFIX + id);
   }
 
   /**
-   * Creates a correlation identifier.
-   * This is a type-safe wrapper over int.
+   * Creates a correlation identifier from a name.
    *
    * @param name     variable name
    */
   public CorrelationId(String name) {
-    assert name != null && name.startsWith(CORREL_PREFIX)
+    this(Integer.parseInt(name.substring(CORREL_PREFIX.length())), name);
+    assert name.startsWith(CORREL_PREFIX)
         : "Correlation name should start with " + CORREL_PREFIX
         + " actual name is " + name;
-    this.id = Integer.parseInt(name.substring(CORREL_PREFIX.length()));
-    this.name = name;
   }
 
   /**
@@ -61,7 +73,7 @@ public class CorrelationId implements Cloneable, Comparable<CorrelationId> {
   }
 
   /**
-   * Returns the preffered name of the variable.
+   * Returns the preferred name of the variable.
    *
    * @return name
    */
@@ -86,6 +98,30 @@ public class CorrelationId implements Cloneable, Comparable<CorrelationId> {
         || obj instanceof CorrelationId
         && this.id == ((CorrelationId) obj).id;
   }
+
+  /** Converts a set of correlation ids to a set of names. */
+  public static ImmutableSet<CorrelationId> setOf(Set<String> set) {
+    if (set.isEmpty()) {
+      return ImmutableSet.of();
+    }
+    final ImmutableSet.Builder<CorrelationId> builder = ImmutableSet.builder();
+    for (String s : set) {
+      builder.add(new CorrelationId(s));
+    }
+    return builder.build();
+  }
+
+  /** Converts a set of names to a set of correlation ids. */
+  public static Set<String> names(Set<CorrelationId> set) {
+    if (set.isEmpty()) {
+      return ImmutableSet.of();
+    }
+    final ImmutableSet.Builder<String> builder = ImmutableSet.builder();
+    for (CorrelationId s : set) {
+      builder.add(s.name);
+    }
+    return builder.build();
+  }
 }
 
 // End CorrelationId.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java b/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java
index 3a06c86..a45a854 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java
@@ -36,13 +36,22 @@ public abstract class EquiJoin extends Join {
   /** Creates an EquiJoin. */
   public EquiJoin(RelOptCluster cluster, RelTraitSet traits, RelNode left,
       RelNode right, RexNode condition, ImmutableIntList leftKeys,
-      ImmutableIntList rightKeys, JoinRelType joinType,
-      Set<String> variablesStopped) {
-    super(cluster, traits, left, right, condition, joinType, variablesStopped);
+      ImmutableIntList rightKeys, Set<CorrelationId> variablesSet,
+      JoinRelType joinType) {
+    super(cluster, traits, left, right, condition, variablesSet, joinType);
     this.leftKeys = Preconditions.checkNotNull(leftKeys);
     this.rightKeys = Preconditions.checkNotNull(rightKeys);
   }
 
+  @Deprecated // to be removed before 2.0
+  public EquiJoin(RelOptCluster cluster, RelTraitSet traits, RelNode left,
+      RelNode right, RexNode condition, ImmutableIntList leftKeys,
+      ImmutableIntList rightKeys, JoinRelType joinType,
+      Set<String> variablesStopped) {
+    this(cluster, traits, left, right, condition, leftKeys, rightKeys,
+        CorrelationId.setOf(variablesStopped), joinType);
+  }
+
   public ImmutableIntList getLeftKeys() {
     return leftKeys;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/core/Join.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Join.java b/core/src/main/java/org/apache/calcite/rel/core/Join.java
index bb460ff..3ac32d2 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Join.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Join.java
@@ -35,6 +35,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
@@ -56,43 +57,67 @@ public abstract class Join extends BiRel {
   //~ Instance fields --------------------------------------------------------
 
   protected final RexNode condition;
-  protected final ImmutableSet<String> variablesStopped;
+  protected final ImmutableSet<CorrelationId> variablesSet;
 
   /**
    * Values must be of enumeration {@link JoinRelType}, except that
    * {@link JoinRelType#RIGHT} is disallowed.
    */
-  protected JoinRelType joinType;
+  protected final JoinRelType joinType;
 
   //~ Constructors -----------------------------------------------------------
 
+  // Next time we need to change the constructor of Join, let's change the
+  // "Set<String> variablesStopped" parameter to
+  // "Set<CorrelationId> variablesSet". At that point we would deprecate
+  // RelNode.getVariablesStopped().
+
   /**
    * Creates a Join.
    *
+   * <p>Note: We plan to change the {@code variablesStopped} parameter to
+   * {@code Set&lt;CorrelationId&gt; variablesSet}
+   * {@link org.apache.calcite.util.Bug#upgrade(String) before version 2.0},
+   * because {@link #getVariablesSet()}
+   * is preferred over {@link #getVariablesStopped()}.
+   * This constructor is not deprecated, for now, because maintaining overloaded
+   * constructors in multiple sub-classes would be onerous.
+   *
    * @param cluster          Cluster
-   * @param traits           Traits
+   * @param traitSet         Trait set
    * @param left             Left input
    * @param right            Right input
    * @param condition        Join condition
    * @param joinType         Join type
-   * @param variablesStopped Set of names of variables which are set by the
+   * @param variablesSet     Set variables that are set by the
    *                         LHS and used by the RHS and are not available to
-   *                         nodes above this LogicalJoin in the tree
+   *                         nodes above this Join in the tree
    */
   protected Join(
       RelOptCluster cluster,
-      RelTraitSet traits,
+      RelTraitSet traitSet,
+      RelNode left,
+      RelNode right,
+      RexNode condition,
+      Set<CorrelationId> variablesSet,
+      JoinRelType joinType) {
+    super(cluster, traitSet, left, right);
+    this.condition = Preconditions.checkNotNull(condition);
+    this.variablesSet = ImmutableSet.copyOf(variablesSet);
+    this.joinType = Preconditions.checkNotNull(joinType);
+  }
+
+  @Deprecated // to be removed before 2.0
+  protected Join(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
       RelNode left,
       RelNode right,
       RexNode condition,
       JoinRelType joinType,
       Set<String> variablesStopped) {
-    super(cluster, traits, left, right);
-    this.condition = condition;
-    this.variablesStopped = ImmutableSet.copyOf(variablesStopped);
-    assert joinType != null;
-    assert condition != null;
-    this.joinType = joinType;
+    this(cluster, traitSet, left, right, condition,
+        CorrelationId.setOf(variablesStopped), joinType);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -172,8 +197,8 @@ public abstract class Join extends BiRel {
     return Util.first(RelMdUtil.getJoinRowCount(this, condition), 1D);
   }
 
-  @Override public Set<String> getVariablesStopped() {
-    return variablesStopped;
+  @Override public Set<CorrelationId> getVariablesSet() {
+    return variablesSet;
   }
 
   @Override public RelWriter explainTerms(RelWriter pw) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 d58150b..6c410b9 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
@@ -24,6 +24,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalIntersect;
 import org.apache.calcite.rel.logical.LogicalJoin;
@@ -37,6 +38,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
@@ -60,6 +62,9 @@ public class RelFactories {
 
   public static final JoinFactory DEFAULT_JOIN_FACTORY = new JoinFactoryImpl();
 
+  public static final CorrelateFactory DEFAULT_CORRELATE_FACTORY =
+      new CorrelateFactoryImpl();
+
   public static final SemiJoinFactory DEFAULT_SEMI_JOIN_FACTORY =
       new SemiJoinFactoryImpl();
 
@@ -234,14 +239,19 @@ public class RelFactories {
      * @param left             Left input
      * @param right            Right input
      * @param condition        Join condition
-     * @param joinType         Join type
-     * @param variablesStopped Set of names of variables which are set by the
+     * @param variablesSet     Set of variables that are set by the
      *                         LHS and used by the RHS and are not available to
      *                         nodes above this LogicalJoin in the tree
+     * @param joinType         Join type
      * @param semiJoinDone     Whether this join has been translated to a
      *                         semi-join
      */
     RelNode createJoin(RelNode left, RelNode right, RexNode condition,
+        Set<CorrelationId> variablesSet, JoinRelType joinType,
+        boolean semiJoinDone);
+
+    @Deprecated // to be removed before 2.0
+    RelNode createJoin(RelNode left, RelNode right, RexNode condition,
         JoinRelType joinType, Set<String> variablesStopped,
         boolean semiJoinDone);
   }
@@ -252,10 +262,51 @@ public class RelFactories {
    */
   private static class JoinFactoryImpl implements JoinFactory {
     public RelNode createJoin(RelNode left, RelNode right,
-        RexNode condition, JoinRelType joinType,
-        Set<String> variablesStopped, boolean semiJoinDone) {
-      return LogicalJoin.create(left, right, condition, joinType,
-          variablesStopped, semiJoinDone, ImmutableList.<RelDataTypeField>of());
+        RexNode condition, Set<CorrelationId> variablesSet,
+        JoinRelType joinType, boolean semiJoinDone) {
+      return LogicalJoin.create(left, right, condition, variablesSet, joinType,
+          semiJoinDone, ImmutableList.<RelDataTypeField>of());
+    }
+
+    public RelNode createJoin(RelNode left, RelNode right, RexNode condition,
+        JoinRelType joinType, Set<String> variablesStopped,
+        boolean semiJoinDone) {
+      return createJoin(left, right, condition,
+          CorrelationId.setOf(variablesStopped), joinType, semiJoinDone);
+    }
+  }
+
+  /**
+   * Can create a correlate of the appropriate type for a rule's calling
+   * convention.
+   *
+   * <p>The result is typically a {@link Correlate}.
+   */
+  public interface CorrelateFactory {
+    /**
+     * Creates a correlate.
+     *
+     * @param left             Left input
+     * @param right            Right input
+     * @param correlationId    Variable name for the row of left input
+     * @param requiredColumns  Required columns
+     * @param joinType         Join type
+     */
+    RelNode createCorrelate(RelNode left, RelNode right,
+        CorrelationId correlationId, ImmutableBitSet requiredColumns,
+        SemiJoinType joinType);
+  }
+
+  /**
+   * Implementation of {@link CorrelateFactory} that returns a vanilla
+   * {@link org.apache.calcite.rel.logical.LogicalCorrelate}.
+   */
+  private static class CorrelateFactoryImpl implements CorrelateFactory {
+    public RelNode createCorrelate(RelNode left, RelNode right,
+        CorrelationId correlationId, ImmutableBitSet requiredColumns,
+        SemiJoinType joinType) {
+      return LogicalCorrelate.create(left, right, correlationId,
+          requiredColumns, joinType);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
index 4570919..6db45f1 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
@@ -71,8 +71,8 @@ public class SemiJoin extends EquiJoin {
         condition,
         leftKeys,
         rightKeys,
-        JoinRelType.INNER,
-        ImmutableSet.<String>of());
+        ImmutableSet.<CorrelationId>of(),
+        JoinRelType.INNER);
   }
 
   /** Creates a SemiJoin. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
index a4da2d4..92324ba 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJson.java
@@ -397,7 +397,7 @@ public class RelJson {
       if (correl != null) {
         final Object jsonType = map.get("type");
         RelDataType type = toType(cluster.getTypeFactory(), jsonType);
-        return rexBuilder.makeCorrel(type, correl);
+        return rexBuilder.makeCorrel(type, new CorrelationId(correl));
       }
       if (map.containsKey("literal")) {
         final Object literal = map.get("literal");

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
index 3fb2123..3ca1645 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
@@ -26,6 +26,7 @@ import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributionTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
 import org.apache.calcite.rel.rules.FilterToCalcRule;
@@ -113,9 +114,9 @@ public final class LogicalCalc extends Calc {
     return new LogicalCalc(getCluster(), traitSet, child, program);
   }
 
-  @Override public void collectVariablesUsed(Set<String> variableSet) {
+  @Override public void collectVariablesUsed(Set<CorrelationId> variableSet) {
     final RelOptUtil.VariableUsedVisitor vuv =
-        new RelOptUtil.VariableUsedVisitor();
+        new RelOptUtil.VariableUsedVisitor(null);
     for (RexNode expr : program.getExprList()) {
       expr.accept(vuv);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
index fd403e0..c0c69d4 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
@@ -19,6 +19,7 @@ package org.apache.calcite.rel.logical;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalcitePrepareImpl;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
@@ -26,6 +27,7 @@ import org.apache.calcite.rel.core.Correlate;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 
 /**
  * A relational operator that performs nested-loop joins.
@@ -69,6 +71,7 @@ public final class LogicalCorrelate extends Correlate {
         correlationId,
         requiredColumns,
         joinType);
+    assert !CalcitePrepareImpl.DEBUG || isValid(Litmus.THROW);
   }
 
   @Deprecated // to be removed before 2.0

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
index 79b0b1d..d25874e 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
@@ -26,20 +26,27 @@ import org.apache.calcite.rel.RelDistributionTraitDef;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
 import org.apache.calcite.rex.RexNode;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableSet;
 
 import java.util.List;
+import java.util.Set;
 
 /**
  * Sub-class of {@link org.apache.calcite.rel.core.Filter}
  * not targeted at any particular engine or calling convention.
  */
 public final class LogicalFilter extends Filter {
+  private final ImmutableSet<CorrelationId> variablesSet;
+
   //~ Constructors -----------------------------------------------------------
 
   /**
@@ -51,21 +58,35 @@ public final class LogicalFilter extends Filter {
    * @param child     Input relational expression
    * @param condition Boolean expression which determines whether a row is
    *                  allowed to pass
+   * @param variablesSet Correlation variables set by this relational expression
+   *                     to be used by nested expressions
    */
   public LogicalFilter(
       RelOptCluster cluster,
       RelTraitSet traitSet,
       RelNode child,
-      RexNode condition) {
+      RexNode condition,
+      ImmutableSet<CorrelationId> variablesSet) {
     super(cluster, traitSet, child, condition);
+    this.variablesSet = Preconditions.checkNotNull(variablesSet);
   }
 
   @Deprecated // to be removed before 2.0
   public LogicalFilter(
       RelOptCluster cluster,
+      RelTraitSet traitSet,
       RelNode child,
       RexNode condition) {
-    this(cluster, cluster.traitSetOf(Convention.NONE), child, condition);
+    this(cluster, traitSet, child, condition, ImmutableSet.<CorrelationId>of());
+  }
+
+  @Deprecated // to be removed before 2.0
+  public LogicalFilter(
+      RelOptCluster cluster,
+      RelNode child,
+      RexNode condition) {
+    this(cluster, cluster.traitSetOf(Convention.NONE), child, condition,
+        ImmutableSet.<CorrelationId>of());
   }
 
   /**
@@ -73,10 +94,17 @@ public final class LogicalFilter extends Filter {
    */
   public LogicalFilter(RelInput input) {
     super(input);
+    this.variablesSet = ImmutableSet.of();
   }
 
   /** Creates a LogicalFilter. */
   public static LogicalFilter create(final RelNode input, RexNode condition) {
+    return create(input, condition, ImmutableSet.<CorrelationId>of());
+  }
+
+  /** Creates a LogicalFilter. */
+  public static LogicalFilter create(final RelNode input, RexNode condition,
+      ImmutableSet<CorrelationId> variablesSet) {
     final RelOptCluster cluster = input.getCluster();
     final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE)
         .replaceIfs(RelCollationTraitDef.INSTANCE,
@@ -91,20 +119,30 @@ public final class LogicalFilter extends Filter {
                 return RelMdDistribution.filter(input);
               }
             });
-    return new LogicalFilter(cluster, traitSet, input, condition);
+    return new LogicalFilter(cluster, traitSet, input, condition, variablesSet);
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  @Override public Set<CorrelationId> getVariablesSet() {
+    return variablesSet;
+  }
+
   public LogicalFilter copy(RelTraitSet traitSet, RelNode input,
       RexNode condition) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new LogicalFilter(getCluster(), traitSet, input, condition);
+    return new LogicalFilter(getCluster(), traitSet, input, condition,
+        variablesSet);
   }
 
   @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
+
+  @Override public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw)
+        .itemIf("variablesSet", variablesSet, !variablesSet.isEmpty());
+  }
 }
 
 // End LogicalFilter.java


[10/50] [abbrv] calcite git commit: RelBuilder.project now does nothing if asked to project the identity with the same field names

Posted by jh...@apache.org.
RelBuilder.project now does nothing if asked to project the identity with the same field names


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

Branch: refs/heads/branch-release
Commit: b5b28f0b2d263a2e2fb6894d67c8666549f4d053
Parents: 4ae0298
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Jul 29 22:01:41 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 core/src/main/java/org/apache/calcite/plan/RelOptUtil.java   | 8 ++++++--
 .../java/org/apache/calcite/plan/SubstitutionVisitor.java    | 3 +--
 .../java/org/apache/calcite/rel/rules/ProjectMergeRule.java  | 5 +++--
 .../java/org/apache/calcite/rel/rules/ProjectRemoveRule.java | 7 ++++---
 core/src/main/java/org/apache/calcite/rex/RexUtil.java       | 7 +++++++
 core/src/main/java/org/apache/calcite/tools/RelBuilder.java  | 6 ++++++
 6 files changed, 27 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/b5b28f0b/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 e444612..46a1dbc 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -42,7 +42,6 @@ import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
 import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.calcite.rel.rules.MultiJoin;
-import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rel.rules.ProjectToWindowRule;
 import org.apache.calcite.rel.rules.PruneEmptyRules;
 import org.apache.calcite.rel.type.RelDataType;
@@ -2706,7 +2705,7 @@ public abstract class RelOptUtil {
             : SqlValidatorUtil.uniquify(fieldNames,
                 SqlValidatorUtil.F_SUGGESTER);
     if (optimize
-        && ProjectRemoveRule.isIdentity(exprs, child.getRowType())) {
+        && RexUtil.isIdentity(exprs, child.getRowType())) {
       if (child instanceof Project && fieldNames != null) {
         final RelDataType rowType =
             RexUtil.createStructType(
@@ -2982,6 +2981,11 @@ public abstract class RelOptUtil {
     final int leftCount = originalJoin.getLeft().getRowType().getFieldCount();
     final int rightCount = originalJoin.getRight().getRowType().getFieldCount();
 
+    // You cannot push a 'get' because field names might change.
+    //
+    // Pushing sub-queries is OK in principle (if they don't reference both
+    // sides of the join via correlating variables) but we'd rather not do it
+    // yet.
     if (!containsGet(joinCond)) {
       joinCond = pushDownEqualJoinConditions(
           joinCond, leftCount, rightCount, extraLeftExprs, extraRightExprs);

http://git-wip-us.apache.org/repos/asf/calcite/blob/b5b28f0b/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 eb0331f..c88f51d 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -37,7 +37,6 @@ import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.logical.LogicalUnion;
-import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
@@ -2144,7 +2143,7 @@ public class SubstitutionVisitor {
     public static boolean isTrivial(MutableProject project) {
       MutableRel child = project.getInput();
       final RelDataType childRowType = child.getRowType();
-      return ProjectRemoveRule.isIdentity(project.getProjects(), childRowType);
+      return RexUtil.isIdentity(project.getProjects(), childRowType);
     }
 
     /** Equivalent to

http://git-wip-us.apache.org/repos/asf/calcite/blob/b5b28f0b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
index 4537157..9a4e849 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
@@ -26,6 +26,7 @@ import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.util.Permutation;
 
 import java.util.List;
@@ -98,7 +99,7 @@ public class ProjectMergeRule extends RelOptRule {
     // If we're not in force mode and the two projects reference identical
     // inputs, then return and let ProjectRemoveRule replace the projects.
     if (!force) {
-      if (ProjectRemoveRule.isIdentity(topProject.getProjects(),
+      if (RexUtil.isIdentity(topProject.getProjects(),
           topProject.getInput().getRowType())) {
         return;
       }
@@ -107,7 +108,7 @@ public class ProjectMergeRule extends RelOptRule {
     final List<RexNode> newProjects =
         RelOptUtil.pushPastProject(topProject.getProjects(), bottomProject);
     final RelNode input = bottomProject.getInput();
-    if (ProjectRemoveRule.isIdentity(newProjects, input.getRowType())) {
+    if (RexUtil.isIdentity(newProjects, input.getRowType())) {
       if (force
           || input.getRowType().getFieldNames()
               .equals(topProject.getRowType().getFieldNames())) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/b5b28f0b/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java
index 81341fc..8cbbd74 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectRemoveRule.java
@@ -85,13 +85,14 @@ public class ProjectRemoveRule extends RelOptRule {
   }
 
   public static boolean isTrivial(Project project) {
-    return isIdentity(project.getProjects(), project.getInput().getRowType());
+    return RexUtil.isIdentity(project.getProjects(),
+        project.getInput().getRowType());
   }
 
+  @Deprecated // to be removed before 1.5
   public static boolean isIdentity(List<? extends RexNode> exps,
       RelDataType childRowType) {
-    return childRowType.getFieldCount() == exps.size()
-        && RexUtil.containIdentity(exps, childRowType, false);
+    return RexUtil.isIdentity(exps, childRowType);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/b5b28f0b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 7307654..d801d1b 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -693,6 +693,13 @@ public class RexUtil {
     return true;
   }
 
+  /** Returns whether a list of expressions projects the incoming fields. */
+  public static boolean isIdentity(List<? extends RexNode> exps,
+      RelDataType inputRowType) {
+    return inputRowType.getFieldCount() == exps.size()
+        && containIdentity(exps, inputRowType, false);
+  }
+
   /**
    * Converts a collection of expressions into an AND.
    * If there are zero expressions, returns TRUE.

http://git-wip-us.apache.org/repos/asf/calcite/blob/b5b28f0b/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 252e703..eb9ad70 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -741,6 +741,12 @@ public class RelBuilder {
     if (ProjectRemoveRule.isIdentity(exprList, peek().getRowType())) {
       return this;
     }
+    final RelDataType inputRowType = peek().getRowType();
+    if (RexUtil.isIdentity(exprList, inputRowType)
+        && names.equals(inputRowType.getFieldNames())) {
+      // Do not create an identity project if it does not rename any fields
+      return this;
+    }
     final RelNode project =
         projectFactory.createProject(build(), ImmutableList.copyOf(exprList),
             names);


[50/50] [abbrv] calcite git commit: [maven-release-plugin] prepare release calcite-1.6.0

Posted by jh...@apache.org.
[maven-release-plugin] prepare release calcite-1.6.0


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

Branch: refs/heads/branch-release
Commit: c4d346b0a413a1a62e028dd3be40071523203a58
Parents: df5ec6d
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 18 12:40:50 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Jan 18 12:40:50 2016 -0800

----------------------------------------------------------------------
 avatica-server/pom.xml   |  4 ++--
 avatica/pom.xml          |  4 ++--
 core/pom.xml             |  4 ++--
 example/csv/pom.xml      |  4 ++--
 example/function/pom.xml |  4 ++--
 example/pom.xml          |  4 ++--
 linq4j/pom.xml           |  4 ++--
 mongodb/pom.xml          |  4 ++--
 piglet/pom.xml           |  4 ++--
 plus/pom.xml             |  4 ++--
 pom.xml                  | 16 ++++++++--------
 spark/pom.xml            |  4 ++--
 splunk/pom.xml           |  4 ++--
 ubenchmark/pom.xml       |  2 +-
 14 files changed, 33 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/avatica-server/pom.xml
----------------------------------------------------------------------
diff --git a/avatica-server/pom.xml b/avatica-server/pom.xml
index d04d2e3..6ee516b 100644
--- a/avatica-server/pom.xml
+++ b/avatica-server/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-avatica-server</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Avatica Server</name>
   <description>JDBC server.</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/avatica/pom.xml
----------------------------------------------------------------------
diff --git a/avatica/pom.xml b/avatica/pom.xml
index fcac174..9a30725 100644
--- a/avatica/pom.xml
+++ b/avatica/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-avatica</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Avatica</name>
   <description>JDBC driver framework.</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 32c6d3a..8aa1b5a 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-core</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Core</name>
   <description>Core Calcite APIs and engine.</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/example/csv/pom.xml
----------------------------------------------------------------------
diff --git a/example/csv/pom.xml b/example/csv/pom.xml
index 1545df9..c9cae00 100644
--- a/example/csv/pom.xml
+++ b/example/csv/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite-example</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-example-csv</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Example CSV</name>
   <description>An example Calcite provider that reads CSV files</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/example/function/pom.xml
----------------------------------------------------------------------
diff --git a/example/function/pom.xml b/example/function/pom.xml
index 1996760..fefa293 100644
--- a/example/function/pom.xml
+++ b/example/function/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite-example</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-example-function</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Example Function</name>
   <description>Examples of user-defined Calcite functions</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/example/pom.xml
----------------------------------------------------------------------
diff --git a/example/pom.xml b/example/pom.xml
index b9684fc..5c4bf84 100644
--- a/example/pom.xml
+++ b/example/pom.xml
@@ -20,13 +20,13 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <!-- The basics. -->
   <artifactId>calcite-example</artifactId>
   <packaging>pom</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Examples</name>
   <description>Calcite examples</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/linq4j/pom.xml
----------------------------------------------------------------------
diff --git a/linq4j/pom.xml b/linq4j/pom.xml
index b9a3c76..d9b4200 100644
--- a/linq4j/pom.xml
+++ b/linq4j/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-linq4j</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Linq4j</name>
   <description>Calcite APIs for LINQ (Language-Integrated Query) in Java</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/mongodb/pom.xml b/mongodb/pom.xml
index 5d23fc2..33953a9 100644
--- a/mongodb/pom.xml
+++ b/mongodb/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-mongodb</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite MongoDB</name>
   <description>MongoDB adapter for Calcite</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/piglet/pom.xml
----------------------------------------------------------------------
diff --git a/piglet/pom.xml b/piglet/pom.xml
index 075842f..d38ecad 100644
--- a/piglet/pom.xml
+++ b/piglet/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-piglet</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Piglet</name>
   <description>Pig-like language built on top of Calcite algebra</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/plus/pom.xml
----------------------------------------------------------------------
diff --git a/plus/pom.xml b/plus/pom.xml
index 99721ce..f413f1f 100644
--- a/plus/pom.xml
+++ b/plus/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-plus</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Plus</name>
   <description>Miscellaneous extras for Calcite</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5c23392..2a5afbd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -27,7 +27,7 @@ limitations under the License.
   <groupId>org.apache.calcite</groupId>
   <artifactId>calcite</artifactId>
   <packaging>pom</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
 
   <!-- More project information. -->
   <name>Calcite</name>
@@ -61,7 +61,7 @@ limitations under the License.
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/calcite.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/calcite.git</developerConnection>
     <url>https://github.com/apache/calcite</url>
-    <tag>HEAD</tag>
+    <tag>calcite-1.6.0</tag>
   </scm>
 
   <modules>
@@ -89,34 +89,34 @@ limitations under the License.
       <dependency>
         <groupId>org.apache.calcite</groupId>
         <artifactId>calcite-avatica</artifactId>
-        <version>1.6.0-SNAPSHOT</version>
+        <version>1.6.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.calcite</groupId>
         <artifactId>calcite-avatica</artifactId>
-        <version>1.6.0-SNAPSHOT</version>
+        <version>1.6.0</version>
         <type>test-jar</type>
       </dependency>
       <dependency>
         <groupId>org.apache.calcite</groupId>
         <artifactId>calcite-avatica-server</artifactId>
-        <version>1.6.0-SNAPSHOT</version>
+        <version>1.6.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.calcite</groupId>
         <artifactId>calcite-core</artifactId>
-        <version>1.6.0-SNAPSHOT</version>
+        <version>1.6.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.calcite</groupId>
         <artifactId>calcite-core</artifactId>
         <type>test-jar</type>
-        <version>1.6.0-SNAPSHOT</version>
+        <version>1.6.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.calcite</groupId>
         <artifactId>calcite-linq4j</artifactId>
-        <version>1.6.0-SNAPSHOT</version>
+        <version>1.6.0</version>
       </dependency>
 
       <!-- Now third-party dependencies, sorted by groupId and artifactId. -->

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/spark/pom.xml
----------------------------------------------------------------------
diff --git a/spark/pom.xml b/spark/pom.xml
index 25448ca..183b8b9 100644
--- a/spark/pom.xml
+++ b/spark/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-spark</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Spark</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/splunk/pom.xml
----------------------------------------------------------------------
diff --git a/splunk/pom.xml b/splunk/pom.xml
index db482d6..9a342b3 100644
--- a/splunk/pom.xml
+++ b/splunk/pom.xml
@@ -20,12 +20,12 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <artifactId>calcite-splunk</artifactId>
   <packaging>jar</packaging>
-  <version>1.6.0-SNAPSHOT</version>
+  <version>1.6.0</version>
   <name>Calcite Splunk</name>
   <description>Splunk adapter for Calcite; also a JDBC driver for Splunk</description>
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c4d346b0/ubenchmark/pom.xml
----------------------------------------------------------------------
diff --git a/ubenchmark/pom.xml b/ubenchmark/pom.xml
index 83e29bb..37c3fb8 100644
--- a/ubenchmark/pom.xml
+++ b/ubenchmark/pom.xml
@@ -20,7 +20,7 @@ limitations under the License.
   <parent>
     <groupId>org.apache.calcite</groupId>
     <artifactId>calcite</artifactId>
-    <version>1.6.0-SNAPSHOT</version>
+    <version>1.6.0</version>
   </parent>
 
   <properties>


[31/50] [abbrv] calcite git commit: [CALCITE-794] Detect cycles when computing statistics

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
index 2a6431d..c872075 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
@@ -44,7 +44,6 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
 
-import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 
 import java.util.ArrayList;
@@ -67,55 +66,53 @@ public class RelMdColumnUniqueness {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Boolean areColumnsUnique(
-      TableScan rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
+  public Boolean areColumnsUnique(TableScan rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
     return rel.getTable().isKey(columns);
   }
 
-  public Boolean areColumnsUnique(
-      Filter rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getInput(),
-        columns,
-        ignoreNulls);
+  public Boolean areColumnsUnique(Filter rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    return mq.areColumnsUnique(rel.getInput(), columns, ignoreNulls);
   }
 
-  public Boolean areColumnsUnique(
-      Sort rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getInput(),
-        columns,
-        ignoreNulls);
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(ImmutableBitSet, boolean)},
+   * invoked using reflection, for any relational expression not
+   * handled by a more specific method.
+   *
+   * @param rel Relational expression
+   * @param mq Metadata query
+   * @param columns column mask representing the subset of columns for which
+   *                uniqueness will be determined
+   * @param ignoreNulls if true, ignore null values when determining column
+   *                    uniqueness
+   * @return whether the columns are unique, or
+   * null if not enough information is available to make that determination
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#areColumnsUnique(RelNode, ImmutableBitSet, boolean)
+   */
+  public Boolean areColumnsUnique(RelNode rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    // no information available
+    return null;
   }
 
-  public Boolean areColumnsUnique(
-      SetOp rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
+  public Boolean areColumnsUnique(SetOp rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
     // If not ALL then the rows are distinct.
     // Therefore the set of all columns is a key.
     return !rel.all
         && columns.nextClearBit(0) >= rel.getRowType().getFieldCount();
   }
 
-  public Boolean areColumnsUnique(
-      Intersect rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    if (areColumnsUnique((SetOp) rel, columns, ignoreNulls)) {
+  public Boolean areColumnsUnique(Intersect rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    if (areColumnsUnique((SetOp) rel, mq, columns, ignoreNulls)) {
       return true;
     }
     for (RelNode input : rel.getInputs()) {
-      Boolean b = RelMetadataQuery.areColumnsUnique(
-          input,
-          columns,
-          ignoreNulls);
+      Boolean b = mq.areColumnsUnique(input, columns, ignoreNulls);
       if (b != null && b) {
         return true;
       }
@@ -123,43 +120,31 @@ public class RelMdColumnUniqueness {
     return false;
   }
 
-  public Boolean areColumnsUnique(
-      Minus rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    if (areColumnsUnique((SetOp) rel, columns, ignoreNulls)) {
+  public Boolean areColumnsUnique(Minus rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    if (areColumnsUnique((SetOp) rel, mq, columns, ignoreNulls)) {
       return true;
     }
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getInput(0),
-        columns,
-        ignoreNulls);
+    return mq.areColumnsUnique(rel.getInput(0), columns, ignoreNulls);
   }
 
-  public Boolean areColumnsUnique(
-      Exchange rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getInput(),
-        columns,
-        ignoreNulls);
+  public Boolean areColumnsUnique(Sort rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    return mq.areColumnsUnique(rel.getInput(), columns, ignoreNulls);
   }
 
-  public Boolean areColumnsUnique(
-      Correlate rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getLeft(),
-        columns,
-        ignoreNulls);
+  public Boolean areColumnsUnique(Exchange rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    return mq.areColumnsUnique(rel.getInput(), columns, ignoreNulls);
+  }
+
+  public Boolean areColumnsUnique(Correlate rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    return mq.areColumnsUnique(rel.getLeft(), columns, ignoreNulls);
   }
 
-  public Boolean areColumnsUnique(
-      Project rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
+  public Boolean areColumnsUnique(Project rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
     // LogicalProject maps a set of rows to a different set;
     // Without knowledge of the mapping function(whether it
     // preserves uniqueness), it is only safe to derive uniqueness
@@ -211,16 +196,12 @@ public class RelMdColumnUniqueness {
       return null;
     }
 
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getInput(),
-        childColumns.build(),
+    return mq.areColumnsUnique(rel.getInput(), childColumns.build(),
         ignoreNulls);
   }
 
-  public Boolean areColumnsUnique(
-      Join rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
+  public Boolean areColumnsUnique(Join rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
     if (columns.cardinality() == 0) {
       return false;
     }
@@ -245,11 +226,9 @@ public class RelMdColumnUniqueness {
     // right hand side, then the columns are unique if and only if they're
     // unique for their respective join inputs
     final ImmutableBitSet leftColumns = leftBuilder.build();
-    Boolean leftUnique =
-        RelMetadataQuery.areColumnsUnique(left, leftColumns, ignoreNulls);
+    Boolean leftUnique = mq.areColumnsUnique(left, leftColumns, ignoreNulls);
     final ImmutableBitSet rightColumns = rightBuilder.build();
-    Boolean rightUnique =
-        RelMetadataQuery.areColumnsUnique(right, rightColumns, ignoreNulls);
+    Boolean rightUnique = mq.areColumnsUnique(right, rightColumns, ignoreNulls);
     if ((leftColumns.cardinality() > 0)
         && (rightColumns.cardinality() > 0)) {
       if ((leftUnique == null) || (rightUnique == null)) {
@@ -271,8 +250,7 @@ public class RelMdColumnUniqueness {
         return false;
       }
       Boolean rightJoinColsUnique =
-          RelMetadataQuery.areColumnsUnique(right, joinInfo.rightSet(),
-              ignoreNulls);
+          mq.areColumnsUnique(right, joinInfo.rightSet(), ignoreNulls);
       if ((rightJoinColsUnique == null) || (leftUnique == null)) {
         return null;
       }
@@ -282,8 +260,7 @@ public class RelMdColumnUniqueness {
         return false;
       }
       Boolean leftJoinColsUnique =
-          RelMetadataQuery.areColumnsUnique(left, joinInfo.leftSet(),
-              ignoreNulls);
+          mq.areColumnsUnique(left, joinInfo.leftSet(), ignoreNulls);
       if ((leftJoinColsUnique == null) || (rightUnique == null)) {
         return null;
       }
@@ -293,56 +270,28 @@ public class RelMdColumnUniqueness {
     throw new AssertionError();
   }
 
-  public Boolean areColumnsUnique(
-      SemiJoin rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
+  public Boolean areColumnsUnique(SemiJoin rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
     // only return the unique keys from the LHS since a semijoin only
     // returns the LHS
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getLeft(),
-        columns,
-        ignoreNulls);
+    return mq.areColumnsUnique(rel.getLeft(), columns, ignoreNulls);
   }
 
-  public Boolean areColumnsUnique(
-      Aggregate rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
+  public Boolean areColumnsUnique(Aggregate rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
     // group by keys form a unique key
     ImmutableBitSet groupKey = ImmutableBitSet.range(rel.getGroupCount());
     return columns.contains(groupKey);
   }
 
-  // Catch-all rule when none of the others apply.
-  public Boolean areColumnsUnique(
-      RelNode rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    // no information available
-    return null;
-  }
-
-  public Boolean areColumnsUnique(
-      Converter rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getInput(),
-        columns,
-        ignoreNulls);
-  }
-
-  public Boolean areColumnsUnique(
-      Values rel,
-      RelMetadataQuery query,
-      boolean ignoreNulls) {
-    if (rel.getTuples().size() < 2) {
+  public Boolean areColumnsUnique(Values rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    if (rel.tuples.size() < 2) {
       return true;
     }
     final Set<List<Comparable>> set = new HashSet<>();
     final List<Comparable> values = new ArrayList<>();
-    for (ImmutableList<RexLiteral> tuple : rel.getTuples()) {
+    for (ImmutableList<RexLiteral> tuple : rel.tuples) {
       for (RexLiteral literal : tuple) {
         values.add(NullSentinel.mask(literal.getValue()));
       }
@@ -354,36 +303,64 @@ public class RelMdColumnUniqueness {
     return true;
   }
 
-  public Boolean areColumnsUnique(
+  public Boolean areColumnsUnique(Converter rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    return mq.areColumnsUnique(rel.getInput(), columns, ignoreNulls);
+  }
+
+  public Boolean areColumnsUnique(HepRelVertex rel, RelMetadataQuery mq,
       boolean dummy, // prevent method from being used
-      HepRelVertex rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    return RelMetadataQuery.areColumnsUnique(
-        rel.getCurrentRel(),
-        columns,
-        ignoreNulls);
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    return mq.areColumnsUnique(rel.getCurrentRel(), columns, ignoreNulls);
   }
 
-  public Boolean areColumnsUnique(
-      RelSubset rel,
-      ImmutableBitSet columns,
-      boolean ignoreNulls) {
-    final RelNode best = rel.getBest();
-    if (best == null) {
-      return null;
-    } else {
-      return RelMetadataQuery.areColumnsUnique(best, columns, ignoreNulls);
+  public Boolean areColumnsUnique(RelSubset rel, RelMetadataQuery mq,
+      ImmutableBitSet columns, boolean ignoreNulls) {
+    int nullCount = 0;
+    for (RelNode rel2 : rel.getRels()) {
+      if (rel2 instanceof Aggregate
+          || rel2 instanceof Filter
+          || rel2 instanceof Values
+          || rel2 instanceof TableScan
+          || simplyProjects(rel2, columns)) {
+        try {
+          final Boolean unique = mq.areColumnsUnique(rel2, columns, ignoreNulls);
+          if (unique != null) {
+            if (unique) {
+              return true;
+            }
+          } else {
+            ++nullCount;
+          }
+        } catch (CyclicMetadataException e) {
+          // Ignore this relational expression; there will be non-cyclic ones
+          // in this set.
+        }
+      }
     }
+    return nullCount == 0 ? false : null;
   }
 
-  /** Aggregate and Calc are "safe" children of a RelSubset to delve into. */
-  private static final Predicate<RelNode> SAFE_REL =
-      new Predicate<RelNode>() {
-        public boolean apply(RelNode r) {
-          return r instanceof Aggregate || r instanceof Project;
-        }
-      };
+  private boolean simplyProjects(RelNode rel, ImmutableBitSet columns) {
+    if (!(rel instanceof Project)) {
+      return false;
+    }
+    Project project = (Project) rel;
+    final List<RexNode> projects = project.getProjects();
+    for (int column : columns) {
+      if (column >= projects.size()) {
+        return false;
+      }
+      if (!(projects.get(column) instanceof RexInputRef)) {
+        return false;
+      }
+      final RexInputRef ref = (RexInputRef) projects.get(column);
+      if (ref.getIndex() != column) {
+        return false;
+      }
+    }
+    return true;
+  }
 }
 
 // End RelMdColumnUniqueness.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
index 2fb2b25..eef245e 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistinctRowCount.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.rel.metadata;
 
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Exchange;
@@ -31,6 +32,7 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.NumberUtil;
@@ -54,10 +56,27 @@ public class RelMdDistinctRowCount {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getDistinctRowCount(
-      Union rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.DistinctRowCount#getDistinctRowCount(ImmutableBitSet, RexNode)},
+   * invoked using reflection.
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#getDistinctRowCount(RelNode, ImmutableBitSet, RexNode)
+   */
+  public Double getDistinctRowCount(RelNode rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
+    // REVIEW zfong 4/19/06 - Broadbase code does not take into
+    // consideration selectivity of predicates passed in.  Also, they
+    // assume the rows are unique even if the table is not
+    boolean uniq = RelMdUtil.areColumnsDefinitelyUnique(mq, rel, groupKey);
+    if (uniq) {
+      return NumberUtil.multiply(mq.getRowCount(rel),
+          mq.getSelectivity(rel, predicate));
+    }
+    return null;
+  }
+
+  public Double getDistinctRowCount(Union rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
     Double rowCount = 0.0;
     int[] adjustments = new int[rel.getRowType().getFieldCount()];
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
@@ -76,10 +95,7 @@ public class RelMdDistinctRowCount {
                     adjustments));
       }
       Double partialRowCount =
-          RelMetadataQuery.getDistinctRowCount(
-              input,
-              groupKey,
-              modifiedPred);
+          mq.getDistinctRowCount(input, groupKey, modifiedPred);
       if (partialRowCount == null) {
         return null;
       }
@@ -88,30 +104,18 @@ public class RelMdDistinctRowCount {
     return rowCount;
   }
 
-  public Double getDistinctRowCount(
-      Sort rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
-    return RelMetadataQuery.getDistinctRowCount(
-        rel.getInput(),
-        groupKey,
-        predicate);
+  public Double getDistinctRowCount(Sort rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
+    return mq.getDistinctRowCount(rel.getInput(), groupKey, predicate);
   }
 
-  public Double getDistinctRowCount(
-      Exchange rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
-    return RelMetadataQuery.getDistinctRowCount(
-        rel.getInput(),
-        groupKey,
-        predicate);
+  public Double getDistinctRowCount(Exchange rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
+    return mq.getDistinctRowCount(rel.getInput(), groupKey, predicate);
   }
 
-  public Double getDistinctRowCount(
-      Filter rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
+  public Double getDistinctRowCount(Filter rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
     if (predicate == null || predicate.isAlwaysTrue()) {
       if (groupKey.isEmpty()) {
         return 1D;
@@ -126,33 +130,22 @@ public class RelMdDistinctRowCount {
             predicate,
             rel.getCondition());
 
-    return RelMetadataQuery.getDistinctRowCount(
-        rel.getInput(),
-        groupKey,
-        unionPreds);
+    return mq.getDistinctRowCount(rel.getInput(), groupKey, unionPreds);
   }
 
-  public Double getDistinctRowCount(
-      Join rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
+  public Double getDistinctRowCount(Join rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
     if (predicate == null || predicate.isAlwaysTrue()) {
       if (groupKey.isEmpty()) {
         return 1D;
       }
     }
-    return RelMdUtil.getJoinDistinctRowCount(
-        rel,
-        rel.getJoinType(),
-        groupKey,
-        predicate,
-        false);
+    return RelMdUtil.getJoinDistinctRowCount(mq, rel, rel.getJoinType(),
+        groupKey, predicate, false);
   }
 
-  public Double getDistinctRowCount(
-      SemiJoin rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
+  public Double getDistinctRowCount(SemiJoin rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
     if (predicate == null || predicate.isAlwaysTrue()) {
       if (groupKey.isEmpty()) {
         return 1D;
@@ -160,7 +153,7 @@ public class RelMdDistinctRowCount {
     }
     // create a RexNode representing the selectivity of the
     // semijoin filter and pass it to getDistinctRowCount
-    RexNode newPred = RelMdUtil.makeSemiJoinSelectivityRexNode(rel);
+    RexNode newPred = RelMdUtil.makeSemiJoinSelectivityRexNode(mq, rel);
     if (predicate != null) {
       RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
       newPred =
@@ -170,16 +163,11 @@ public class RelMdDistinctRowCount {
               predicate);
     }
 
-    return RelMetadataQuery.getDistinctRowCount(
-        rel.getLeft(),
-        groupKey,
-        newPred);
+    return mq.getDistinctRowCount(rel.getLeft(), groupKey, newPred);
   }
 
-  public Double getDistinctRowCount(
-      Aggregate rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
+  public Double getDistinctRowCount(Aggregate rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
     if (predicate == null || predicate.isAlwaysTrue()) {
       if (groupKey.isEmpty()) {
         return 1D;
@@ -187,8 +175,8 @@ public class RelMdDistinctRowCount {
     }
     // determine which predicates can be applied on the child of the
     // aggregate
-    List<RexNode> notPushable = new ArrayList<RexNode>();
-    List<RexNode> pushable = new ArrayList<RexNode>();
+    final List<RexNode> notPushable = new ArrayList<>();
+    final List<RexNode> pushable = new ArrayList<>();
     RelOptUtil.splitFilters(
         rel.getGroupSet(),
         predicate,
@@ -203,10 +191,7 @@ public class RelMdDistinctRowCount {
     RelMdUtil.setAggChildKeys(groupKey, rel, childKey);
 
     Double distinctRowCount =
-        RelMetadataQuery.getDistinctRowCount(
-            rel.getInput(),
-            childKey.build(),
-            childPreds);
+        mq.getDistinctRowCount(rel.getInput(), childKey.build(), childPreds);
     if (distinctRowCount == null) {
       return null;
     } else if (notPushable.isEmpty()) {
@@ -218,10 +203,8 @@ public class RelMdDistinctRowCount {
     }
   }
 
-  public Double getDistinctRowCount(
-      Values rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
+  public Double getDistinctRowCount(Values rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
     if (predicate == null || predicate.isAlwaysTrue()) {
       if (groupKey.isEmpty()) {
         return 1D;
@@ -230,14 +213,12 @@ public class RelMdDistinctRowCount {
     Double selectivity = RelMdUtil.guessSelectivity(predicate);
 
     // assume half the rows are duplicates
-    Double nRows = rel.getRows() / 2;
+    Double nRows = rel.estimateRowCount(mq) / 2;
     return RelMdUtil.numDistinctVals(nRows, nRows * selectivity);
   }
 
-  public Double getDistinctRowCount(
-      Project rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
+  public Double getDistinctRowCount(Project rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
     if (predicate == null || predicate.isAlwaysTrue()) {
       if (groupKey.isEmpty()) {
         return 1D;
@@ -248,8 +229,8 @@ public class RelMdDistinctRowCount {
     List<RexNode> projExprs = rel.getProjects();
     RelMdUtil.splitCols(projExprs, groupKey, baseCols, projCols);
 
-    List<RexNode> notPushable = new ArrayList<RexNode>();
-    List<RexNode> pushable = new ArrayList<RexNode>();
+    final List<RexNode> notPushable = new ArrayList<>();
+    final List<RexNode> pushable = new ArrayList<>();
     RelOptUtil.splitFilters(
         ImmutableBitSet.range(rel.getRowType().getFieldCount()),
         predicate,
@@ -269,9 +250,7 @@ public class RelMdDistinctRowCount {
       modifiedPred = RelOptUtil.pushPastProject(childPred, rel);
     }
     Double distinctRowCount =
-        RelMetadataQuery.getDistinctRowCount(
-            rel.getInput(),
-            baseCols.build(),
+        mq.getDistinctRowCount(rel.getInput(), baseCols.build(),
             modifiedPred);
 
     if (distinctRowCount == null) {
@@ -291,38 +270,36 @@ public class RelMdDistinctRowCount {
     // multiply by the cardinality of the non-child projection expressions
     for (int bit : projCols.build()) {
       Double subRowCount =
-          RelMdUtil.cardOfProjExpr(rel, projExprs.get(bit));
+          RelMdUtil.cardOfProjExpr(mq, rel, projExprs.get(bit));
       if (subRowCount == null) {
         return null;
       }
       distinctRowCount *= subRowCount;
     }
 
-    return RelMdUtil.numDistinctVals(
-        distinctRowCount,
-        RelMetadataQuery.getRowCount(rel));
+    return RelMdUtil.numDistinctVals(distinctRowCount, mq.getRowCount(rel));
   }
 
-  // Catch-all rule when none of the others apply.
-  public Double getDistinctRowCount(
-      RelNode rel,
-      ImmutableBitSet groupKey,
-      RexNode predicate) {
-    if (predicate == null || predicate.isAlwaysTrue()) {
-      if (groupKey.isEmpty()) {
-        return 1D;
-      }
+  public Double getDistinctRowCount(RelSubset rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
+    final RelNode best = rel.getBest();
+    if (best != null) {
+      return mq.getDistinctRowCount(best, groupKey, predicate);
     }
-    // REVIEW zfong 4/19/06 - Broadbase code does not take into
-    // consideration selectivity of predicates passed in.  Also, they
-    // assume the rows are unique even if the table is not
-    boolean uniq = RelMdUtil.areColumnsDefinitelyUnique(rel, groupKey);
-    if (uniq) {
-      return NumberUtil.multiply(
-          RelMetadataQuery.getRowCount(rel),
-          RelMetadataQuery.getSelectivity(rel, predicate));
+    if (!Bug.CALCITE_1048_FIXED) {
+      return getDistinctRowCount((RelNode) rel, mq, groupKey, predicate);
     }
-    return null;
+    Double d = null;
+    for (RelNode r2 : rel.getRels()) {
+      try {
+        Double d2 = mq.getDistinctRowCount(r2, groupKey, predicate);
+        d = NumberUtil.min(d, d2);
+      } catch (CyclicMetadataException e) {
+        // Ignore this relational expression; there will be non-cyclic ones
+        // in this set.
+      }
+    }
+    return d;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistribution.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistribution.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistribution.java
index 67ac6a2..f7b5c83 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistribution.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdDistribution.java
@@ -63,40 +63,40 @@ public class RelMdDistribution {
    * @param rel Relational expression
    * @return Relational expression's distribution
    */
-  public RelDistribution distribution(RelNode rel) {
+  public RelDistribution distribution(RelMetadataQuery mq, RelNode rel) {
     return RelDistributions.SINGLETON;
   }
 
-  public RelDistribution distribution(SingleRel rel) {
-    return RelMetadataQuery.distribution(rel.getInput());
+  public RelDistribution distribution(RelMetadataQuery mq, SingleRel rel) {
+    return mq.distribution(rel.getInput());
   }
 
-  public RelDistribution distribution(BiRel rel) {
-    return RelMetadataQuery.distribution(rel.getLeft());
+  public RelDistribution distribution(RelMetadataQuery mq, BiRel rel) {
+    return mq.distribution(rel.getLeft());
   }
 
-  public RelDistribution distribution(SetOp rel) {
-    return RelMetadataQuery.distribution(rel.getInputs().get(0));
+  public RelDistribution distribution(RelMetadataQuery mq, SetOp rel) {
+    return mq.distribution(rel.getInputs().get(0));
   }
 
-  public RelDistribution distribution(TableScan scan) {
+  public RelDistribution distribution(RelMetadataQuery mq, TableScan scan) {
     return table(scan.getTable());
   }
 
-  public RelDistribution distribution(Project project) {
-    return project(project.getInput(), project.getProjects());
+  public RelDistribution distribution(RelMetadataQuery mq, Project project) {
+    return project(mq, project.getInput(), project.getProjects());
   }
 
-  public RelDistribution distribution(Values values) {
+  public RelDistribution distribution(RelMetadataQuery mq, Values values) {
     return values(values.getRowType(), values.getTuples());
   }
 
-  public RelDistribution distribution(Exchange exchange) {
+  public RelDistribution distribution(RelMetadataQuery mq, Exchange exchange) {
     return exchange(exchange.distribution);
   }
 
-  public RelDistribution distribution(HepRelVertex rel) {
-    return RelMetadataQuery.distribution(rel.getCurrentRel());
+  public RelDistribution distribution(RelMetadataQuery mq, HepRelVertex rel) {
+    return mq.distribution(rel.getCurrentRel());
   }
 
   // Helper methods
@@ -109,34 +109,33 @@ public class RelMdDistribution {
 
   /** Helper method to determine a
    * {@link Sort}'s distribution. */
-  public static RelDistribution sort(RelNode input) {
-    return RelMetadataQuery.distribution(input);
+  public static RelDistribution sort(RelMetadataQuery mq, RelNode input) {
+    return mq.distribution(input);
   }
 
   /** Helper method to determine a
    * {@link Filter}'s distribution. */
-  public static RelDistribution filter(RelNode input) {
-    return RelMetadataQuery.distribution(input);
+  public static RelDistribution filter(RelMetadataQuery mq, RelNode input) {
+    return mq.distribution(input);
   }
 
   /** Helper method to determine a
    * limit's distribution. */
-  public static RelDistribution limit(RelNode input) {
-    return RelMetadataQuery.distribution(input);
+  public static RelDistribution limit(RelMetadataQuery mq, RelNode input) {
+    return mq.distribution(input);
   }
 
   /** Helper method to determine a
    * {@link org.apache.calcite.rel.core.Calc}'s distribution. */
-  public static RelDistribution calc(RelNode input,
+  public static RelDistribution calc(RelMetadataQuery mq, RelNode input,
       RexProgram program) {
     throw new AssertionError(); // TODO:
   }
 
   /** Helper method to determine a {@link Project}'s collation. */
-  public static RelDistribution project(RelNode input,
+  public static RelDistribution project(RelMetadataQuery mq, RelNode input,
       List<? extends RexNode> projects) {
-    final RelDistribution inputDistribution =
-        RelMetadataQuery.distribution(input);
+    final RelDistribution inputDistribution = mq.distribution(input);
     final Mappings.TargetMapping mapping =
         Project.getPartialMapping(input.getRowType().getFieldCount(),
             projects);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java
index b0d926e..bf51f2f 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExplainVisibility.java
@@ -36,8 +36,14 @@ public class RelMdExplainVisibility {
 
   //~ Methods ----------------------------------------------------------------
 
-  // Catch-all rule when none of the others apply.
-  public Boolean isVisibleInExplain(RelNode rel, SqlExplainLevel explainLevel) {
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.ExplainVisibility#isVisibleInExplain(SqlExplainLevel)},
+   * invoked using reflection.
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#isVisibleInExplain(RelNode, SqlExplainLevel)
+   */
+  public Boolean isVisibleInExplain(RelNode rel, RelMetadataQuery mq,
+      SqlExplainLevel explainLevel) {
     // no information available
     return null;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMaxRowCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMaxRowCount.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMaxRowCount.java
index 1d2a378..ab9739f 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMaxRowCount.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMaxRowCount.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.rel.metadata;
 
+import org.apache.calcite.adapter.enumerable.EnumerableLimit;
 import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
@@ -29,7 +30,9 @@ import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
 
 /**
  * RelMdMaxRowCount supplies a default implementation of
@@ -42,10 +45,10 @@ public class RelMdMaxRowCount {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getMaxRowCount(Union rel) {
+  public Double getMaxRowCount(Union rel, RelMetadataQuery mq) {
     double rowCount = 0.0;
     for (RelNode input : rel.getInputs()) {
-      Double partialRowCount = RelMetadataQuery.getMaxRowCount(input);
+      Double partialRowCount = mq.getMaxRowCount(input);
       if (partialRowCount == null) {
         return null;
       }
@@ -54,11 +57,11 @@ public class RelMdMaxRowCount {
     return rowCount;
   }
 
-  public Double getMaxRowCount(Intersect rel) {
+  public Double getMaxRowCount(Intersect rel, RelMetadataQuery mq) {
     // max row count is the smallest of the inputs
     Double rowCount = null;
     for (RelNode input : rel.getInputs()) {
-      Double partialRowCount = RelMetadataQuery.getMaxRowCount(input);
+      Double partialRowCount = mq.getMaxRowCount(input);
       if (rowCount == null
           || partialRowCount != null && partialRowCount < rowCount) {
         rowCount = partialRowCount;
@@ -67,22 +70,39 @@ public class RelMdMaxRowCount {
     return rowCount;
   }
 
-  public Double getMaxRowCount(Minus rel) {
-    return RelMetadataQuery.getMaxRowCount(rel.getInput(0));
+  public Double getMaxRowCount(Minus rel, RelMetadataQuery mq) {
+    return mq.getMaxRowCount(rel.getInput(0));
   }
 
-  public Double getMaxRowCount(Filter rel) {
-    return RelMetadataQuery.getMaxRowCount(rel.getInput());
+  public Double getMaxRowCount(Filter rel, RelMetadataQuery mq) {
+    return mq.getMaxRowCount(rel.getInput());
   }
 
-  public Double getMaxRowCount(Project rel) {
-    return RelMetadataQuery.getMaxRowCount(rel.getInput());
+  public Double getMaxRowCount(Project rel, RelMetadataQuery mq) {
+    return mq.getMaxRowCount(rel.getInput());
   }
 
-  public Double getMaxRowCount(Sort rel) {
-    Double rowCount = RelMetadataQuery.getMaxRowCount(rel.getInput());
+  public Double getMaxRowCount(Sort rel, RelMetadataQuery mq) {
+    Double rowCount = mq.getMaxRowCount(rel.getInput());
     if (rowCount == null) {
-      return null;
+      rowCount = Double.POSITIVE_INFINITY;
+    }
+    final int offset = rel.offset == null ? 0 : RexLiteral.intValue(rel.offset);
+    rowCount = Math.max(rowCount - offset, 0D);
+
+    if (rel.fetch != null) {
+      final int limit = RexLiteral.intValue(rel.fetch);
+      if (limit < rowCount) {
+        return (double) limit;
+      }
+    }
+    return rowCount;
+  }
+
+  public Double getMaxRowCount(EnumerableLimit rel, RelMetadataQuery mq) {
+    Double rowCount = mq.getMaxRowCount(rel.getInput());
+    if (rowCount == null) {
+      rowCount = Double.POSITIVE_INFINITY;
     }
     final int offset = rel.offset == null ? 0 : RexLiteral.intValue(rel.offset);
     rowCount = Math.max(rowCount - offset, 0D);
@@ -96,21 +116,21 @@ public class RelMdMaxRowCount {
     return rowCount;
   }
 
-  public Double getMaxRowCount(Aggregate rel) {
+  public Double getMaxRowCount(Aggregate rel, RelMetadataQuery mq) {
     if (rel.getGroupSet().isEmpty()) {
       // Aggregate with no GROUP BY always returns 1 row (even on empty table).
       return 1D;
     }
-    final Double rowCount = RelMetadataQuery.getMaxRowCount(rel.getInput());
+    final Double rowCount = mq.getMaxRowCount(rel.getInput());
     if (rowCount == null) {
       return null;
     }
     return rowCount * rel.getGroupSets().size();
   }
 
-  public Double getMaxRowCount(Join rel) {
-    Double left = RelMetadataQuery.getMaxRowCount(rel.getLeft());
-    Double right = RelMetadataQuery.getMaxRowCount(rel.getRight());
+  public Double getMaxRowCount(Join rel, RelMetadataQuery mq) {
+    Double left = mq.getMaxRowCount(rel.getLeft());
+    Double right = mq.getMaxRowCount(rel.getRight());
     if (left == null || right == null) {
       return null;
     }
@@ -123,20 +143,21 @@ public class RelMdMaxRowCount {
     return left * right;
   }
 
-  public Double getMaxRowCount(TableScan rel) {
+  public Double getMaxRowCount(TableScan rel, RelMetadataQuery mq) {
     // For typical tables, there is no upper bound to the number of rows.
     return Double.POSITIVE_INFINITY;
   }
 
-  public Double getMaxRowCount(Values values) {
+  public Double getMaxRowCount(Values values, RelMetadataQuery mq) {
     // For Values, the maximum row count is the actual row count.
     // This is especially useful if Values is empty.
     return (double) values.getTuples().size();
   }
 
-  public Double getMaxRowCount(RelSubset rel) {
-    // FIXME This is a short-term fix for CALCITE-1018. A complete
-    // solution will come with CALCITE-794.
+  public Double getMaxRowCount(RelSubset rel, RelMetadataQuery mq) {
+    // FIXME This is a short-term fix for [CALCITE-1018]. A complete
+    // solution will come with [CALCITE-1048].
+    Util.discard(Bug.CALCITE_1048_FIXED);
     for (RelNode node : rel.getRels()) {
       if (node instanceof Sort) {
         Sort sort = (Sort) node;
@@ -150,7 +171,7 @@ public class RelMdMaxRowCount {
   }
 
   // Catch-all rule when none of the others apply.
-  public Double getMaxRowCount(RelNode rel) {
+  public Double getMaxRowCount(RelNode rel, RelMetadataQuery mq) {
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMemory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMemory.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMemory.java
index be53b5c..c760b0e 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMemory.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdMemory.java
@@ -43,34 +43,34 @@ public class RelMdMemory {
   //~ Methods ----------------------------------------------------------------
 
   /** Catch-all implementation for
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Memory#memory()},
+   * {@link BuiltInMetadata.Memory#memory()},
    * invoked using reflection.
    *
    * @see org.apache.calcite.rel.metadata.RelMetadataQuery#memory
    */
-  public Double memory(RelNode rel) {
+  public Double memory(RelNode rel, RelMetadataQuery mq) {
     return null;
   }
 
   /** Catch-all implementation for
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Memory#cumulativeMemoryWithinPhase()},
+   * {@link BuiltInMetadata.Memory#cumulativeMemoryWithinPhase()},
    * invoked using reflection.
    *
    * @see org.apache.calcite.rel.metadata.RelMetadataQuery#memory
    */
-  public Double cumulativeMemoryWithinPhase(RelNode rel) {
-    Double nullable = RelMetadataQuery.memory(rel);
+  public Double cumulativeMemoryWithinPhase(RelNode rel, RelMetadataQuery mq) {
+    Double nullable = mq.memory(rel);
     if (nullable == null) {
       return null;
     }
-    Boolean isPhaseTransition = RelMetadataQuery.isPhaseTransition(rel);
+    Boolean isPhaseTransition = mq.isPhaseTransition(rel);
     if (isPhaseTransition == null) {
       return null;
     }
     double d = nullable;
     if (!isPhaseTransition) {
       for (RelNode input : rel.getInputs()) {
-        nullable = RelMetadataQuery.cumulativeMemoryWithinPhase(input);
+        nullable = mq.cumulativeMemoryWithinPhase(input);
         if (nullable == null) {
           return null;
         }
@@ -81,15 +81,15 @@ public class RelMdMemory {
   }
 
   /** Catch-all implementation for
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Memory#cumulativeMemoryWithinPhaseSplit()},
+   * {@link BuiltInMetadata.Memory#cumulativeMemoryWithinPhaseSplit()},
    * invoked using reflection.
    *
    * @see org.apache.calcite.rel.metadata.RelMetadataQuery#cumulativeMemoryWithinPhaseSplit
    */
-  public Double cumulativeMemoryWithinPhaseSplit(RelNode rel) {
-    final Double memoryWithinPhase =
-        RelMetadataQuery.cumulativeMemoryWithinPhase(rel);
-    final Integer splitCount = RelMetadataQuery.splitCount(rel);
+  public Double cumulativeMemoryWithinPhaseSplit(RelNode rel,
+      RelMetadataQuery mq) {
+    final Double memoryWithinPhase = mq.cumulativeMemoryWithinPhase(rel);
+    final Integer splitCount = mq.splitCount(rel);
     if (memoryWithinPhase == null || splitCount == null) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdParallelism.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdParallelism.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdParallelism.java
index ca9c244..dcd69f1 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdParallelism.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdParallelism.java
@@ -45,34 +45,34 @@ public class RelMdParallelism {
   //~ Methods ----------------------------------------------------------------
 
   /** Catch-all implementation for
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Parallelism#isPhaseTransition()},
+   * {@link BuiltInMetadata.Parallelism#isPhaseTransition()},
    * invoked using reflection.
    *
    * @see org.apache.calcite.rel.metadata.RelMetadataQuery#isPhaseTransition
    */
-  public Boolean isPhaseTransition(RelNode rel) {
+  public Boolean isPhaseTransition(RelNode rel, RelMetadataQuery mq) {
     return false;
   }
 
-  public Boolean isPhaseTransition(TableScan rel) {
+  public Boolean isPhaseTransition(TableScan rel, RelMetadataQuery mq) {
     return true;
   }
 
-  public Boolean isPhaseTransition(Values rel) {
+  public Boolean isPhaseTransition(Values rel, RelMetadataQuery mq) {
     return true;
   }
 
-  public Boolean isPhaseTransition(Exchange rel) {
+  public Boolean isPhaseTransition(Exchange rel, RelMetadataQuery mq) {
     return true;
   }
 
   /** Catch-all implementation for
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Parallelism#splitCount()},
+   * {@link BuiltInMetadata.Parallelism#splitCount()},
    * invoked using reflection.
    *
    * @see org.apache.calcite.rel.metadata.RelMetadataQuery#splitCount
    */
-  public Integer splitCount(RelNode rel) {
+  public Integer splitCount(RelNode rel, RelMetadataQuery mq) {
     return 1;
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java
index 991a488..bde07a9 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPercentageOriginalRows.java
@@ -53,16 +53,14 @@ public class RelMdPercentageOriginalRows {
 
   private RelMdPercentageOriginalRows() {}
 
-  public Double getPercentageOriginalRows(Aggregate rel) {
+  public Double getPercentageOriginalRows(Aggregate rel, RelMetadataQuery mq) {
     // REVIEW jvs 28-Mar-2006: The assumption here seems to be that
     // aggregation does not apply any filtering, so it does not modify the
     // percentage.  That's very much oversimplified.
-
-    return RelMetadataQuery.getPercentageOriginalRows(
-        rel.getInput());
+    return mq.getPercentageOriginalRows(rel.getInput());
   }
 
-  public Double getPercentageOriginalRows(Union rel) {
+  public Double getPercentageOriginalRows(Union rel, RelMetadataQuery mq) {
     double numerator = 0.0;
     double denominator = 0.0;
 
@@ -77,9 +75,8 @@ public class RelMdPercentageOriginalRows {
     // case where a huge table has been completely filtered away.
 
     for (RelNode input : rel.getInputs()) {
-      double rowCount = RelMetadataQuery.getRowCount(input);
-      double percentage =
-          RelMetadataQuery.getPercentageOriginalRows(input);
+      double rowCount = mq.getRowCount(input);
+      double percentage = mq.getPercentageOriginalRows(input);
       if (percentage != 0.0) {
         denominator += rowCount / percentage;
         numerator += rowCount;
@@ -89,7 +86,7 @@ public class RelMdPercentageOriginalRows {
     return quotientForPercentage(numerator, denominator);
   }
 
-  public Double getPercentageOriginalRows(Join rel) {
+  public Double getPercentageOriginalRows(Join rel, RelMetadataQuery mq) {
     // Assume any single-table filter conditions have already
     // been pushed down.
 
@@ -98,16 +95,13 @@ public class RelMdPercentageOriginalRows {
 
     // REVIEW jvs 28-Mar-2006:  need any special casing for SemiJoin?
 
-    double left = RelMetadataQuery.getPercentageOriginalRows(rel.getLeft());
-
-    double right =
-        RelMetadataQuery.getPercentageOriginalRows(rel.getRight());
-
+    double left = mq.getPercentageOriginalRows(rel.getLeft());
+    double right = mq.getPercentageOriginalRows(rel.getRight());
     return left * right;
   }
 
   // Catch-all rule when none of the others apply.
-  public Double getPercentageOriginalRows(RelNode rel) {
+  public Double getPercentageOriginalRows(RelNode rel, RelMetadataQuery mq) {
     if (rel.getInputs().size() > 1) {
       // No generic formula available for multiple inputs.
       return null;
@@ -120,8 +114,7 @@ public class RelMdPercentageOriginalRows {
 
     RelNode child = rel.getInputs().get(0);
 
-    Double childPercentage =
-        RelMetadataQuery.getPercentageOriginalRows(child);
+    Double childPercentage = mq.getPercentageOriginalRows(child);
     if (childPercentage == null) {
       return null;
     }
@@ -130,9 +123,7 @@ public class RelMdPercentageOriginalRows {
     // filtering is the effect of single-table filters) with the percentage
     // filtering performed by the child.
     Double relPercentage =
-        quotientForPercentage(
-            RelMetadataQuery.getRowCount(rel),
-            RelMetadataQuery.getRowCount(child));
+        quotientForPercentage(mq.getRowCount(rel), mq.getRowCount(child));
     if (relPercentage == null) {
       return null;
     }
@@ -147,22 +138,23 @@ public class RelMdPercentageOriginalRows {
   }
 
   // Ditto for getNonCumulativeCost
-  public RelOptCost getCumulativeCost(RelNode rel) {
-    RelOptCost cost = RelMetadataQuery.getNonCumulativeCost(rel);
+  public RelOptCost getCumulativeCost(RelNode rel, RelMetadataQuery mq) {
+    RelOptCost cost = mq.getNonCumulativeCost(rel);
     List<RelNode> inputs = rel.getInputs();
     for (RelNode input : inputs) {
-      cost = cost.plus(RelMetadataQuery.getCumulativeCost(input));
+      cost = cost.plus(mq.getCumulativeCost(input));
     }
     return cost;
   }
 
-  public RelOptCost getCumulativeCost(EnumerableInterpreter rel) {
-    return RelMetadataQuery.getNonCumulativeCost(rel);
+  public RelOptCost getCumulativeCost(EnumerableInterpreter rel,
+      RelMetadataQuery mq) {
+    return mq.getNonCumulativeCost(rel);
   }
 
   // Ditto for getNonCumulativeCost
-  public RelOptCost getNonCumulativeCost(RelNode rel) {
-    return rel.computeSelfCost(rel.getCluster().getPlanner());
+  public RelOptCost getNonCumulativeCost(RelNode rel, RelMetadataQuery mq) {
+    return rel.computeSelfCost(rel.getCluster().getPlanner(), mq);
   }
 
   private static Double quotientForPercentage(

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
index 47c0fcc..180883f 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPopulationSize.java
@@ -47,28 +47,26 @@ public class RelMdPopulationSize {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getPopulationSize(Filter rel, ImmutableBitSet groupKey) {
-    return RelMetadataQuery.getPopulationSize(
-        rel.getInput(),
-        groupKey);
+  public Double getPopulationSize(Filter rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
+    return mq.getPopulationSize(rel.getInput(), groupKey);
   }
 
-  public Double getPopulationSize(Sort rel, ImmutableBitSet groupKey) {
-    return RelMetadataQuery.getPopulationSize(
-        rel.getInput(),
-        groupKey);
+  public Double getPopulationSize(Sort rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
+    return mq.getPopulationSize(rel.getInput(), groupKey);
   }
 
-  public Double getPopulationSize(Exchange rel, ImmutableBitSet groupKey) {
-    return RelMetadataQuery.getPopulationSize(
-        rel.getInput(),
-        groupKey);
+  public Double getPopulationSize(Exchange rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
+    return mq.getPopulationSize(rel.getInput(), groupKey);
   }
 
-  public Double getPopulationSize(Union rel, ImmutableBitSet groupKey) {
+  public Double getPopulationSize(Union rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
     Double population = 0.0;
     for (RelNode input : rel.getInputs()) {
-      Double subPop = RelMetadataQuery.getPopulationSize(input, groupKey);
+      Double subPop = mq.getPopulationSize(input, groupKey);
       if (subPop == null) {
         return null;
       }
@@ -77,35 +75,38 @@ public class RelMdPopulationSize {
     return population;
   }
 
-  public Double getPopulationSize(Join rel, ImmutableBitSet groupKey) {
-    return RelMdUtil.getJoinPopulationSize(rel, groupKey);
+  public Double getPopulationSize(Join rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
+    return RelMdUtil.getJoinPopulationSize(mq, rel, groupKey);
   }
 
-  public Double getPopulationSize(SemiJoin rel, ImmutableBitSet groupKey) {
-    return RelMetadataQuery.getPopulationSize(rel.getLeft(), groupKey);
+  public Double getPopulationSize(SemiJoin rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
+    return mq.getPopulationSize(rel.getLeft(), groupKey);
   }
 
-  public Double getPopulationSize(Aggregate rel, ImmutableBitSet groupKey) {
+  public Double getPopulationSize(Aggregate rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
     ImmutableBitSet.Builder childKey = ImmutableBitSet.builder();
     RelMdUtil.setAggChildKeys(groupKey, rel, childKey);
-    return RelMetadataQuery.getPopulationSize(rel.getInput(), childKey.build());
+    return mq.getPopulationSize(rel.getInput(), childKey.build());
   }
 
-  public Double getPopulationSize(Values rel, ImmutableBitSet groupKey) {
+  public Double getPopulationSize(Values rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
     // assume half the rows are duplicates
-    return rel.getRows() / 2;
+    return rel.estimateRowCount(mq) / 2;
   }
 
-  public Double getPopulationSize(Project rel, ImmutableBitSet groupKey) {
+  public Double getPopulationSize(Project rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
     ImmutableBitSet.Builder baseCols = ImmutableBitSet.builder();
     ImmutableBitSet.Builder projCols = ImmutableBitSet.builder();
     List<RexNode> projExprs = rel.getProjects();
     RelMdUtil.splitCols(projExprs, groupKey, baseCols, projCols);
 
     Double population =
-        RelMetadataQuery.getPopulationSize(
-            rel.getInput(),
-            baseCols.build());
+        mq.getPopulationSize(rel.getInput(), baseCols.build());
     if (population == null) {
       return null;
     }
@@ -118,7 +119,7 @@ public class RelMdPopulationSize {
 
     for (int bit : projCols.build()) {
       Double subRowCount =
-          RelMdUtil.cardOfProjExpr(rel, projExprs.get(bit));
+          RelMdUtil.cardOfProjExpr(mq, rel, projExprs.get(bit));
       if (subRowCount == null) {
         return null;
       }
@@ -128,22 +129,26 @@ public class RelMdPopulationSize {
     // REVIEW zfong 6/22/06 - Broadbase did not have the call to
     // numDistinctVals.  This is needed; otherwise, population can be
     // larger than the number of rows in the RelNode.
-    return RelMdUtil.numDistinctVals(
-        population,
-        RelMetadataQuery.getRowCount(rel));
+    return RelMdUtil.numDistinctVals(population, mq.getRowCount(rel));
   }
 
-  // Catch-all rule when none of the others apply.
-  public Double getPopulationSize(RelNode rel, ImmutableBitSet groupKey) {
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.PopulationSize#getPopulationSize(ImmutableBitSet)},
+   * invoked using reflection.
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#getPopulationSize(RelNode, ImmutableBitSet)
+   */
+  public Double getPopulationSize(RelNode rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey) {
     // if the keys are unique, return the row count; otherwise, we have
     // no further information on which to return any legitimate value
 
     // REVIEW zfong 4/11/06 - Broadbase code returns the product of each
     // unique key, which would result in the population being larger
     // than the total rows in the relnode
-    boolean uniq = RelMdUtil.areColumnsDefinitelyUnique(rel, groupKey);
+    boolean uniq = RelMdUtil.areColumnsDefinitelyUnique(mq, rel, groupKey);
     if (uniq) {
-      return RelMetadataQuery.getRowCount(rel);
+      return mq.getRowCount(rel);
     }
 
     return null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
index 2119abf..4b1a894 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
@@ -21,6 +21,7 @@ import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Predicate1;
 import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.hep.HepRelVertex;
 import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
@@ -120,15 +121,26 @@ public class RelMdPredicates {
 
   private static final List<RexNode> EMPTY_LIST = ImmutableList.of();
 
-  // Catch-all rule when none of the others apply.
-  public RelOptPredicateList getPredicates(RelNode rel) {
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.Predicates#getPredicates()},
+   * invoked using reflection.
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#getPulledUpPredicates(RelNode)
+   */
+  public RelOptPredicateList getPredicates(RelNode rel, RelMetadataQuery mq) {
     return RelOptPredicateList.EMPTY;
   }
 
+  public RelOptPredicateList getPredicates(HepRelVertex rel,
+      RelMetadataQuery mq) {
+    return mq.getPulledUpPredicates(rel.getCurrentRel());
+  }
+
   /**
    * Infers predicates for a table scan.
    */
-  public RelOptPredicateList getPredicates(TableScan table) {
+  public RelOptPredicateList getPredicates(TableScan table,
+      RelMetadataQuery mq) {
     return RelOptPredicateList.EMPTY;
   }
 
@@ -144,24 +156,23 @@ public class RelMdPredicates {
    * is not in the projection list.
    *
    * <pre>
-   * childPullUpExprs:      {a &gt; 7, b + c &lt; 10, a + e = 9}
+   * inputPullUpExprs:      {a &gt; 7, b + c &lt; 10, a + e = 9}
    * projectionExprs:       {a, b, c, e / 2}
    * projectionPullupExprs: {a &gt; 7, b + c &lt; 10}
    * </pre>
    *
    * </ol>
    */
-  public RelOptPredicateList getPredicates(Project project) {
-    RelNode child = project.getInput();
+  public RelOptPredicateList getPredicates(Project project,
+      RelMetadataQuery mq) {
+    final RelNode input = project.getInput();
     final RexBuilder rexBuilder = project.getCluster().getRexBuilder();
-    RelOptPredicateList childInfo =
-        RelMetadataQuery.getPulledUpPredicates(child);
-
-    List<RexNode> projectPullUpPredicates = new ArrayList<RexNode>();
+    final RelOptPredicateList inputInfo = mq.getPulledUpPredicates(input);
+    final List<RexNode> projectPullUpPredicates = new ArrayList<>();
 
     ImmutableBitSet.Builder columnsMappedBuilder = ImmutableBitSet.builder();
     Mapping m = Mappings.create(MappingType.PARTIAL_FUNCTION,
-        child.getRowType().getFieldCount(),
+        input.getRowType().getFieldCount(),
         project.getRowType().getFieldCount());
 
     for (Ord<RexNode> o : Ord.zip(project.getProjects())) {
@@ -175,10 +186,10 @@ public class RelMdPredicates {
     // Go over childPullUpPredicates. If a predicate only contains columns in
     // 'columnsMapped' construct a new predicate based on mapping.
     final ImmutableBitSet columnsMapped = columnsMappedBuilder.build();
-    for (RexNode r : childInfo.pulledUpPredicates) {
+    for (RexNode r : inputInfo.pulledUpPredicates) {
       ImmutableBitSet rCols = RelOptUtil.InputFinder.bits(r);
       if (columnsMapped.contains(rCols)) {
-        r = r.accept(new RexPermuteInputsShuttle(m, child));
+        r = r.accept(new RexPermuteInputsShuttle(m, input));
         projectPullUpPredicates.add(r);
       }
     }
@@ -203,29 +214,27 @@ public class RelMdPredicates {
   }
 
   /**
-   * Add the Filter condition to the pulledPredicates list from the child.
+   * Add the Filter condition to the pulledPredicates list from the input.
    */
-  public RelOptPredicateList getPredicates(Filter filter) {
-    RelNode child = filter.getInput();
-    RelOptPredicateList childInfo =
-        RelMetadataQuery.getPulledUpPredicates(child);
+  public RelOptPredicateList getPredicates(Filter filter, RelMetadataQuery mq) {
+    final RelNode input = filter.getInput();
+    final RelOptPredicateList inputInfo = mq.getPulledUpPredicates(input);
 
-    return Util.first(childInfo, RelOptPredicateList.EMPTY)
+    return Util.first(inputInfo, RelOptPredicateList.EMPTY)
         .union(
             RelOptPredicateList.of(
                 RelOptUtil.conjunctions(filter.getCondition())));
   }
 
   /** Infers predicates for a {@link org.apache.calcite.rel.core.SemiJoin}. */
-  public RelOptPredicateList getPredicates(SemiJoin semiJoin) {
+  public RelOptPredicateList getPredicates(SemiJoin semiJoin,
+      RelMetadataQuery mq) {
     RexBuilder rB = semiJoin.getCluster().getRexBuilder();
-    RelNode left = semiJoin.getInput(0);
-    RelNode right = semiJoin.getInput(1);
+    final RelNode left = semiJoin.getInput(0);
+    final RelNode right = semiJoin.getInput(1);
 
-    RelOptPredicateList leftInfo =
-        RelMetadataQuery.getPulledUpPredicates(left);
-    RelOptPredicateList rightInfo =
-        RelMetadataQuery.getPulledUpPredicates(right);
+    final RelOptPredicateList leftInfo = mq.getPulledUpPredicates(left);
+    final RelOptPredicateList rightInfo = mq.getPulledUpPredicates(right);
 
     JoinConditionBasedPredicateInference jI =
         new JoinConditionBasedPredicateInference(semiJoin,
@@ -236,15 +245,13 @@ public class RelMdPredicates {
   }
 
   /** Infers predicates for a {@link org.apache.calcite.rel.core.Join}. */
-  public RelOptPredicateList getPredicates(Join join) {
+  public RelOptPredicateList getPredicates(Join join, RelMetadataQuery mq) {
     RexBuilder rB = join.getCluster().getRexBuilder();
     RelNode left = join.getInput(0);
     RelNode right = join.getInput(1);
 
-    RelOptPredicateList leftInfo =
-        RelMetadataQuery.getPulledUpPredicates(left);
-    RelOptPredicateList rightInfo =
-        RelMetadataQuery.getPulledUpPredicates(right);
+    final RelOptPredicateList leftInfo = mq.getPulledUpPredicates(left);
+    final RelOptPredicateList rightInfo = mq.getPulledUpPredicates(right);
 
     JoinConditionBasedPredicateInference jI =
         new JoinConditionBasedPredicateInference(join,
@@ -262,31 +269,29 @@ public class RelMdPredicates {
    * GroupSet. For e.g.
    *
    * <pre>
-   * childPullUpExprs : { a &gt; 7, b + c &lt; 10, a + e = 9}
+   * inputPullUpExprs : { a &gt; 7, b + c &lt; 10, a + e = 9}
    * groupSet         : { a, b}
    * pulledUpExprs    : { a &gt; 7}
    * </pre>
    */
-  public RelOptPredicateList getPredicates(Aggregate agg) {
-    RelNode child = agg.getInput();
-    RelOptPredicateList childInfo =
-        RelMetadataQuery.getPulledUpPredicates(child);
-
-    List<RexNode> aggPullUpPredicates = new ArrayList<RexNode>();
+  public RelOptPredicateList getPredicates(Aggregate agg, RelMetadataQuery mq) {
+    final RelNode input = agg.getInput();
+    final RelOptPredicateList inputInfo = mq.getPulledUpPredicates(input);
+    final List<RexNode> aggPullUpPredicates = new ArrayList<>();
 
     ImmutableBitSet groupKeys = agg.getGroupSet();
     Mapping m = Mappings.create(MappingType.PARTIAL_FUNCTION,
-        child.getRowType().getFieldCount(), agg.getRowType().getFieldCount());
+        input.getRowType().getFieldCount(), agg.getRowType().getFieldCount());
 
     int i = 0;
     for (int j : groupKeys) {
       m.set(j, i++);
     }
 
-    for (RexNode r : childInfo.pulledUpPredicates) {
+    for (RexNode r : inputInfo.pulledUpPredicates) {
       ImmutableBitSet rCols = RelOptUtil.InputFinder.bits(r);
       if (groupKeys.contains(rCols)) {
-        r = r.accept(new RexPermuteInputsShuttle(m, child));
+        r = r.accept(new RexPermuteInputsShuttle(m, input));
         aggPullUpPredicates.add(r);
       }
     }
@@ -298,11 +303,11 @@ public class RelMdPredicates {
    *
    * <p>The pulled up expression is a disjunction of its children's predicates.
    */
-  public RelOptPredicateList getPredicates(Union union) {
+  public RelOptPredicateList getPredicates(Union union, RelMetadataQuery mq) {
     RexBuilder rB = union.getCluster().getRexBuilder();
     List<RexNode> orList = Lists.newArrayList();
     for (RelNode input : union.getInputs()) {
-      RelOptPredicateList info = RelMetadataQuery.getPulledUpPredicates(input);
+      RelOptPredicateList info = mq.getPulledUpPredicates(input);
       if (info.pulledUpPredicates.isEmpty()) {
         return RelOptPredicateList.EMPTY;
       }
@@ -321,27 +326,29 @@ public class RelMdPredicates {
   /**
    * Infers predicates for a Sort.
    */
-  public RelOptPredicateList getPredicates(Sort sort) {
-    RelNode child = sort.getInput();
-    return RelMetadataQuery.getPulledUpPredicates(child);
+  public RelOptPredicateList getPredicates(Sort sort, RelMetadataQuery mq) {
+    RelNode input = sort.getInput();
+    return mq.getPulledUpPredicates(input);
   }
 
   /**
    * Infers predicates for an Exchange.
    */
-  public RelOptPredicateList getPredicates(Exchange exchange) {
-    RelNode child = exchange.getInput();
-    return RelMetadataQuery.getPulledUpPredicates(child);
+  public RelOptPredicateList getPredicates(Exchange exchange,
+      RelMetadataQuery mq) {
+    RelNode input = exchange.getInput();
+    return mq.getPulledUpPredicates(input);
   }
 
   /** @see RelMetadataQuery#getPulledUpPredicates(RelNode) */
-  public RelOptPredicateList getPredicates(RelSubset r) {
-    if (!Bug.CALCITE_794_FIXED) {
+  public RelOptPredicateList getPredicates(RelSubset r,
+      RelMetadataQuery mq) {
+    if (!Bug.CALCITE_1048_FIXED) {
       return RelOptPredicateList.EMPTY;
     }
     RelOptPredicateList list = null;
     for (RelNode r2 : r.getRels()) {
-      RelOptPredicateList list2 = RelMetadataQuery.getPulledUpPredicates(r2);
+      RelOptPredicateList list2 = mq.getPulledUpPredicates(r2);
       if (list2 != null) {
         list = list == null ? list2 : list.union(list2);
       }
@@ -409,7 +416,7 @@ public class RelMdPredicates {
           nSysFields + nFieldsLeft + nFieldsRight);
 
       exprFields = Maps.newHashMap();
-      allExprsDigests = new HashSet<String>();
+      allExprsDigests = new HashSet<>();
 
       if (lPreds == null) {
         leftChildPredicates = null;
@@ -440,7 +447,7 @@ public class RelMdPredicates {
       }
 
       equivalence = Maps.newTreeMap();
-      equalityPredicates = new HashSet<String>();
+      equalityPredicates = new HashSet<>();
       for (int i = 0; i < nSysFields + nFieldsLeft + nFieldsRight; i++) {
         equivalence.put(i, BitSets.of(i));
       }
@@ -454,11 +461,13 @@ public class RelMdPredicates {
               compose(rexBuilder, ImmutableList.of(joinRel.getCondition())));
 
       final EquivalenceFinder eF = new EquivalenceFinder();
-      new ArrayList<Void>(Lists.transform(exprs, new Function<RexNode, Void>() {
-        public Void apply(RexNode input) {
-          return input.accept(eF);
-        }
-      }));
+      new ArrayList<>(
+          Lists.transform(exprs,
+              new Function<RexNode, Void>() {
+                public Void apply(RexNode input) {
+                  return input.accept(eF);
+                }
+              }));
 
       equivalence = BitSets.closure(equivalence);
     }
@@ -478,8 +487,8 @@ public class RelMdPredicates {
      */
     public RelOptPredicateList inferPredicates(
         boolean includeEqualityInference) {
-      List<RexNode> inferredPredicates = new ArrayList<RexNode>();
-      Set<String> allExprsDigests = new HashSet<String>(this.allExprsDigests);
+      final List<RexNode> inferredPredicates = new ArrayList<>();
+      final Set<String> allExprsDigests = new HashSet<>(this.allExprsDigests);
       final JoinRelType joinType = joinRel.getJoinType();
       switch (joinType) {
       case INNER:
@@ -509,9 +518,8 @@ public class RelMdPredicates {
           nSysFields + nFieldsLeft, 0, nSysFields, nFieldsLeft);
       final RexPermuteInputsShuttle leftPermute =
           new RexPermuteInputsShuttle(leftMapping, joinRel);
-
-      List<RexNode> leftInferredPredicates = new ArrayList<RexNode>();
-      List<RexNode> rightInferredPredicates = new ArrayList<RexNode>();
+      final List<RexNode> leftInferredPredicates = new ArrayList<>();
+      final List<RexNode> rightInferredPredicates = new ArrayList<>();
 
       for (RexNode iP : inferredPredicates) {
         ImmutableBitSet iPBitSet = RelOptUtil.InputFinder.bits(iP);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
index aac3aae..ea647a6 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdRowCount.java
@@ -16,20 +16,29 @@
  */
 package org.apache.calcite.rel.metadata;
 
+import org.apache.calcite.adapter.enumerable.EnumerableLimit;
+import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Calc;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.Minus;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.SemiJoin;
 import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.NumberUtil;
+import org.apache.calcite.util.Util;
 
 /**
  * RelMdRowCount supplies a default implementation of
@@ -42,10 +51,37 @@ public class RelMdRowCount {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getRowCount(Union rel) {
-    double rowCount = 0D;
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.RowCount#getRowCount()},
+   * invoked using reflection.
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#getRowCount(RelNode)
+   */
+  public Double getRowCount(RelNode rel, RelMetadataQuery mq) {
+    return rel.estimateRowCount(mq);
+  }
+
+  public Double getRowCount(RelSubset subset, RelMetadataQuery mq) {
+    if (!Bug.CALCITE_1048_FIXED) {
+      return mq.getRowCount(Util.first(subset.getBest(), subset.getOriginal()));
+    }
+    Double v = null;
+    for (RelNode r : subset.getRels()) {
+      try {
+        v = NumberUtil.min(v, mq.getRowCount(r));
+      } catch (CyclicMetadataException e) {
+        // ignore this rel; there will be other, non-cyclic ones
+      } catch (Throwable e) {
+        e.printStackTrace();
+      }
+    }
+    return Util.first(v, 1e6d); // if set is empty, estimate large
+  }
+
+  public Double getRowCount(Union rel, RelMetadataQuery mq) {
+    double rowCount = 0.0;
     for (RelNode input : rel.getInputs()) {
-      Double partialRowCount = RelMetadataQuery.getRowCount(input);
+      Double partialRowCount = mq.getRowCount(input);
       if (partialRowCount == null) {
         return null;
       }
@@ -54,10 +90,10 @@ public class RelMdRowCount {
     return rowCount;
   }
 
-  public Double getRowCount(Intersect rel) {
+  public Double getRowCount(Intersect rel, RelMetadataQuery mq) {
     Double rowCount = null;
     for (RelNode input : rel.getInputs()) {
-      Double partialRowCount = RelMetadataQuery.getRowCount(input);
+      Double partialRowCount = mq.getRowCount(input);
       if (rowCount == null
           || partialRowCount != null && partialRowCount < rowCount) {
         rowCount = partialRowCount;
@@ -66,10 +102,10 @@ public class RelMdRowCount {
     return rowCount;
   }
 
-  public Double getRowCount(Minus rel) {
+  public Double getRowCount(Minus rel, RelMetadataQuery mq) {
     Double rowCount = null;
     for (RelNode input : rel.getInputs()) {
-      Double partialRowCount = RelMetadataQuery.getRowCount(input);
+      Double partialRowCount = mq.getRowCount(input);
       if (rowCount == null
           || partialRowCount != null && partialRowCount < rowCount) {
         rowCount = partialRowCount;
@@ -78,20 +114,21 @@ public class RelMdRowCount {
     return rowCount;
   }
 
-  public Double getRowCount(Filter rel) {
-    return NumberUtil.multiply(
-        RelMetadataQuery.getSelectivity(
-            rel.getInput(),
-            rel.getCondition()),
-        RelMetadataQuery.getRowCount(rel.getInput()));
+  public Double getRowCount(Filter rel, RelMetadataQuery mq) {
+    return RelMdUtil.estimateFilteredRows(rel.getInput(), rel.getCondition(),
+        mq);
   }
 
-  public Double getRowCount(Project rel) {
-    return RelMetadataQuery.getRowCount(rel.getInput());
+  public Double getRowCount(Calc rel, RelMetadataQuery mq) {
+    return RelMdUtil.estimateFilteredRows(rel.getInput(), rel.getProgram(), mq);
   }
 
-  public Double getRowCount(Sort rel) {
-    Double rowCount = RelMetadataQuery.getRowCount(rel.getInput());
+  public Double getRowCount(Project rel, RelMetadataQuery mq) {
+    return mq.getRowCount(rel.getInput());
+  }
+
+  public Double getRowCount(Sort rel, RelMetadataQuery mq) {
+    Double rowCount = mq.getRowCount(rel.getInput());
     if (rowCount == null) {
       return null;
     }
@@ -107,30 +144,51 @@ public class RelMdRowCount {
     return rowCount;
   }
 
-  public Double getRowCount(SemiJoin rel) {
+  public Double getRowCount(EnumerableLimit rel, RelMetadataQuery mq) {
+    Double rowCount = mq.getRowCount(rel.getInput());
+    if (rowCount == null) {
+      return null;
+    }
+    final int offset = rel.offset == null ? 0 : RexLiteral.intValue(rel.offset);
+    rowCount = Math.max(rowCount - offset, 0D);
+
+    if (rel.fetch != null) {
+      final int limit = RexLiteral.intValue(rel.fetch);
+      if (limit < rowCount) {
+        return (double) limit;
+      }
+    }
+    return rowCount;
+  }
+
+  // Covers Converter, Interpreter
+  public Double getRowCount(SingleRel rel, RelMetadataQuery mq) {
+    return mq.getRowCount(rel.getInput());
+  }
+
+  public Double getRowCount(Join rel, RelMetadataQuery mq) {
+    return RelMdUtil.getJoinRowCount(mq, rel, rel.getCondition());
+  }
+
+  public Double getRowCount(SemiJoin rel, RelMetadataQuery mq) {
     // create a RexNode representing the selectivity of the
     // semijoin filter and pass it to getSelectivity
     RexNode semiJoinSelectivity =
-        RelMdUtil.makeSemiJoinSelectivityRexNode(rel);
+        RelMdUtil.makeSemiJoinSelectivityRexNode(mq, rel);
 
     return NumberUtil.multiply(
-        RelMetadataQuery.getSelectivity(
-            rel.getLeft(),
-            semiJoinSelectivity),
-        RelMetadataQuery.getRowCount(rel.getLeft()));
+        mq.getSelectivity(rel.getLeft(), semiJoinSelectivity),
+        mq.getRowCount(rel.getLeft()));
   }
 
-  public Double getRowCount(Aggregate rel) {
-    ImmutableBitSet groupKey = rel.getGroupSet();
+  public Double getRowCount(Aggregate rel, RelMetadataQuery mq) {
+    ImmutableBitSet groupKey = rel.getGroupSet(); // .range(rel.getGroupCount());
 
-    // rowcount is the cardinality of the group by columns
+    // rowCount is the cardinality of the group by columns
     Double distinctRowCount =
-        RelMetadataQuery.getDistinctRowCount(
-            rel.getInput(),
-            groupKey,
-            null);
+        mq.getDistinctRowCount(rel.getInput(), groupKey, null);
     if (distinctRowCount == null) {
-      distinctRowCount = RelMetadataQuery.getRowCount(rel.getInput()) / 10D;
+      distinctRowCount = mq.getRowCount(rel.getInput()) / 10;
     }
 
     // Grouping sets multiply
@@ -139,9 +197,12 @@ public class RelMdRowCount {
     return distinctRowCount;
   }
 
-  // Catch-all rule when none of the others apply.
-  public Double getRowCount(RelNode rel) {
-    return rel.getRows();
+  public Double getRowCount(TableScan rel, RelMetadataQuery mq) {
+    return rel.estimateRowCount(mq);
+  }
+
+  public Double getRowCount(Values rel, RelMetadataQuery mq) {
+    return rel.estimateRowCount(mq);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
index c59a005..b7ffcdd 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSelectivity.java
@@ -50,7 +50,8 @@ public class RelMdSelectivity {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Double getSelectivity(Union rel, RexNode predicate) {
+  public Double getSelectivity(Union rel, RelMetadataQuery mq,
+      RexNode predicate) {
     if ((rel.getInputs().size() == 0) || (predicate == null)) {
       return 1.0;
     }
@@ -60,7 +61,7 @@ public class RelMdSelectivity {
     int[] adjustments = new int[rel.getRowType().getFieldCount()];
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
     for (RelNode input : rel.getInputs()) {
-      Double nRows = RelMetadataQuery.getRowCount(input);
+      Double nRows = mq.getRowCount(input);
       if (nRows == null) {
         return null;
       }
@@ -73,7 +74,7 @@ public class RelMdSelectivity {
                   null,
                   input.getRowType().getFieldList(),
                   adjustments));
-      double sel = RelMetadataQuery.getSelectivity(input, modifiedPred);
+      double sel = mq.getSelectivity(input, modifiedPred);
 
       sumRows += nRows;
       sumSelectedRows += nRows * sel;
@@ -85,36 +86,34 @@ public class RelMdSelectivity {
     return sumSelectedRows / sumRows;
   }
 
-  public Double getSelectivity(Sort rel, RexNode predicate) {
-    return RelMetadataQuery.getSelectivity(
-        rel.getInput(),
-        predicate);
+  public Double getSelectivity(Sort rel, RelMetadataQuery mq,
+      RexNode predicate) {
+    return mq.getSelectivity(rel.getInput(), predicate);
   }
 
-  public Double getSelectivity(Filter rel, RexNode predicate) {
+  public Double getSelectivity(Filter rel, RelMetadataQuery mq,
+      RexNode predicate) {
     // Take the difference between the predicate passed in and the
     // predicate in the filter's condition, so we don't apply the
     // selectivity of the filter twice.  If no predicate is passed in,
     // use the filter's condition.
     if (predicate != null) {
-      return RelMetadataQuery.getSelectivity(
-          rel.getInput(),
+      return mq.getSelectivity(rel.getInput(),
           RelMdUtil.minusPreds(
               rel.getCluster().getRexBuilder(),
               predicate,
               rel.getCondition()));
     } else {
-      return RelMetadataQuery.getSelectivity(
-          rel.getInput(),
-          rel.getCondition());
+      return mq.getSelectivity(rel.getInput(), rel.getCondition());
     }
   }
 
-  public Double getSelectivity(SemiJoin rel, RexNode predicate) {
+  public Double getSelectivity(SemiJoin rel, RelMetadataQuery mq,
+      RexNode predicate) {
     // create a RexNode representing the selectivity of the
     // semijoin filter and pass it to getSelectivity
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
-    RexNode newPred = RelMdUtil.makeSemiJoinSelectivityRexNode(rel);
+    RexNode newPred = RelMdUtil.makeSemiJoinSelectivityRexNode(mq, rel);
     if (predicate != null) {
       newPred =
           rexBuilder.makeCall(
@@ -123,14 +122,13 @@ public class RelMdSelectivity {
               predicate);
     }
 
-    return RelMetadataQuery.getSelectivity(
-        rel.getLeft(),
-        newPred);
+    return mq.getSelectivity(rel.getLeft(), newPred);
   }
 
-  public Double getSelectivity(Aggregate rel, RexNode predicate) {
-    List<RexNode> notPushable = new ArrayList<RexNode>();
-    List<RexNode> pushable = new ArrayList<RexNode>();
+  public Double getSelectivity(Aggregate rel, RelMetadataQuery mq,
+      RexNode predicate) {
+    final List<RexNode> notPushable = new ArrayList<>();
+    final List<RexNode> pushable = new ArrayList<>();
     RelOptUtil.splitFilters(
         rel.getGroupSet(),
         predicate,
@@ -140,10 +138,7 @@ public class RelMdSelectivity {
     RexNode childPred =
         RexUtil.composeConjunction(rexBuilder, pushable, true);
 
-    Double selectivity =
-        RelMetadataQuery.getSelectivity(
-            rel.getInput(),
-            childPred);
+    Double selectivity = mq.getSelectivity(rel.getInput(), childPred);
     if (selectivity == null) {
       return null;
     } else {
@@ -153,9 +148,10 @@ public class RelMdSelectivity {
     }
   }
 
-  public Double getSelectivity(Project rel, RexNode predicate) {
-    List<RexNode> notPushable = new ArrayList<RexNode>();
-    List<RexNode> pushable = new ArrayList<RexNode>();
+  public Double getSelectivity(Project rel, RelMetadataQuery mq,
+      RexNode predicate) {
+    final List<RexNode> notPushable = new ArrayList<>();
+    final List<RexNode> pushable = new ArrayList<>();
     RelOptUtil.splitFilters(
         ImmutableBitSet.range(rel.getRowType().getFieldCount()),
         predicate,
@@ -171,10 +167,7 @@ public class RelMdSelectivity {
     } else {
       modifiedPred = RelOptUtil.pushPastProject(childPred, rel);
     }
-    Double selectivity =
-        RelMetadataQuery.getSelectivity(
-            rel.getInput(),
-            modifiedPred);
+    Double selectivity = mq.getSelectivity(rel.getInput(), modifiedPred);
     if (selectivity == null) {
       return null;
     } else {
@@ -185,7 +178,8 @@ public class RelMdSelectivity {
   }
 
   // Catch-all rule when none of the others apply.
-  public Double getSelectivity(RelNode rel, RexNode predicate) {
+  public Double getSelectivity(RelNode rel, RelMetadataQuery mq,
+      RexNode predicate) {
     return RelMdUtil.guessSelectivity(predicate);
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java
index af60e73..8c2502e 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java
@@ -74,14 +74,13 @@ public class RelMdSize {
   //~ Methods ----------------------------------------------------------------
 
   /** Catch-all implementation for
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Size#averageRowSize()},
+   * {@link BuiltInMetadata.Size#averageRowSize()},
    * invoked using reflection.
    *
    * @see org.apache.calcite.rel.metadata.RelMetadataQuery#getAverageRowSize
    */
-  public Double averageRowSize(RelNode rel) {
-    final List<Double> averageColumnSizes =
-        RelMetadataQuery.getAverageColumnSizes(rel);
+  public Double averageRowSize(RelNode rel, RelMetadataQuery mq) {
+    final List<Double> averageColumnSizes = mq.getAverageColumnSizes(rel);
     if (averageColumnSizes == null) {
       return null;
     }
@@ -99,30 +98,30 @@ public class RelMdSize {
   }
 
   /** Catch-all implementation for
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Size#averageColumnSizes()},
+   * {@link BuiltInMetadata.Size#averageColumnSizes()},
    * invoked using reflection.
    *
    * @see org.apache.calcite.rel.metadata.RelMetadataQuery#getAverageColumnSizes
    */
-  public List<Double> averageColumnSizes(RelNode rel) {
+  public List<Double> averageColumnSizes(RelNode rel, RelMetadataQuery mq) {
     return null; // absolutely no idea
   }
 
-  public List<Double> averageColumnSizes(Filter rel) {
-    return RelMetadataQuery.getAverageColumnSizes(rel.getInput());
+  public List<Double> averageColumnSizes(Filter rel, RelMetadataQuery mq) {
+    return mq.getAverageColumnSizes(rel.getInput());
   }
 
-  public List<Double> averageColumnSizes(Sort rel) {
-    return RelMetadataQuery.getAverageColumnSizes(rel.getInput());
+  public List<Double> averageColumnSizes(Sort rel, RelMetadataQuery mq) {
+    return mq.getAverageColumnSizes(rel.getInput());
   }
 
-  public List<Double> averageColumnSizes(Exchange rel) {
-    return RelMetadataQuery.getAverageColumnSizes(rel.getInput());
+  public List<Double> averageColumnSizes(Exchange rel, RelMetadataQuery mq) {
+    return mq.getAverageColumnSizes(rel.getInput());
   }
 
-  public List<Double> averageColumnSizes(Project rel) {
+  public List<Double> averageColumnSizes(Project rel, RelMetadataQuery mq) {
     final List<Double> inputColumnSizes =
-        RelMetadataQuery.getAverageColumnSizesNotNull(rel.getInput());
+        mq.getAverageColumnSizesNotNull(rel.getInput());
     final ImmutableNullableList.Builder<Double> sizes =
         ImmutableNullableList.builder();
     for (RexNode project : rel.getProjects()) {
@@ -131,7 +130,7 @@ public class RelMdSize {
     return sizes.build();
   }
 
-  public List<Double> averageColumnSizes(Values rel) {
+  public List<Double> averageColumnSizes(Values rel, RelMetadataQuery mq) {
     final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
     final ImmutableList.Builder<Double> list = ImmutableList.builder();
     for (int i = 0; i < fields.size(); i++) {
@@ -151,7 +150,7 @@ public class RelMdSize {
     return list.build();
   }
 
-  public List<Double> averageColumnSizes(TableScan rel) {
+  public List<Double> averageColumnSizes(TableScan rel, RelMetadataQuery mq) {
     final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
     final ImmutableList.Builder<Double> list = ImmutableList.builder();
     for (RelDataTypeField field : fields) {
@@ -160,9 +159,9 @@ public class RelMdSize {
     return list.build();
   }
 
-  public List<Double> averageColumnSizes(Aggregate rel) {
+  public List<Double> averageColumnSizes(Aggregate rel, RelMetadataQuery mq) {
     final List<Double> inputColumnSizes =
-        RelMetadataQuery.getAverageColumnSizesNotNull(rel.getInput());
+        mq.getAverageColumnSizesNotNull(rel.getInput());
     final ImmutableList.Builder<Double> list = ImmutableList.builder();
     for (int key : rel.getGroupSet()) {
       list.add(inputColumnSizes.get(key));
@@ -173,21 +172,21 @@ public class RelMdSize {
     return list.build();
   }
 
-  public List<Double> averageColumnSizes(SemiJoin rel) {
-    return averageJoinColumnSizes(rel, true);
+  public List<Double> averageColumnSizes(SemiJoin rel, RelMetadataQuery mq) {
+    return averageJoinColumnSizes(rel, mq, true);
   }
 
-  public List<Double> averageColumnSizes(Join rel) {
-    return averageJoinColumnSizes(rel, false);
+  public List<Double> averageColumnSizes(Join rel, RelMetadataQuery mq) {
+    return averageJoinColumnSizes(rel, mq, false);
   }
 
-  private List<Double> averageJoinColumnSizes(Join rel, boolean semijoin) {
+  private List<Double> averageJoinColumnSizes(Join rel, RelMetadataQuery mq,
+      boolean semijoin) {
     final RelNode left = rel.getLeft();
     final RelNode right = rel.getRight();
-    final List<Double> lefts =
-        RelMetadataQuery.getAverageColumnSizes(left);
-    final List<Double> rights = semijoin
-        ? null : RelMetadataQuery.getAverageColumnSizes(right);
+    final List<Double> lefts = mq.getAverageColumnSizes(left);
+    final List<Double> rights =
+        semijoin ? null : mq.getAverageColumnSizes(right);
     if (lefts == null && rights == null) {
       return null;
     }
@@ -205,20 +204,19 @@ public class RelMdSize {
     return ImmutableNullableList.copyOf(sizes);
   }
 
-  public List<Double> averageColumnSizes(Intersect rel) {
-    return RelMetadataQuery.getAverageColumnSizes(rel.getInput(0));
+  public List<Double> averageColumnSizes(Intersect rel, RelMetadataQuery mq) {
+    return mq.getAverageColumnSizes(rel.getInput(0));
   }
 
-  public List<Double> averageColumnSizes(Minus rel) {
-    return RelMetadataQuery.getAverageColumnSizes(rel.getInput(0));
+  public List<Double> averageColumnSizes(Minus rel, RelMetadataQuery mq) {
+    return mq.getAverageColumnSizes(rel.getInput(0));
   }
 
-  public List<Double> averageColumnSizes(Union rel) {
+  public List<Double> averageColumnSizes(Union rel, RelMetadataQuery mq) {
     final int fieldCount = rel.getRowType().getFieldCount();
     List<List<Double>> inputColumnSizeList = Lists.newArrayList();
     for (RelNode input : rel.getInputs()) {
-      final List<Double> inputSizes =
-          RelMetadataQuery.getAverageColumnSizes(input);
+      final List<Double> inputSizes = mq.getAverageColumnSizes(input);
       if (inputSizes != null) {
         inputColumnSizeList.add(inputSizes);
       }


[49/50] [abbrv] calcite git commit: [CALCITE-1060] Fix test deadlock by initializing DriverManager before registering AlternatingDriver

Posted by jh...@apache.org.
[CALCITE-1060] Fix test deadlock by initializing DriverManager before registering AlternatingDriver

Early DriverManager initialization is required to prevent
Driver.<clinit> -> DriverManager.findServiceProviders -> Class.forName("Driver")
deadlock.

Note: proper resolution might need to add an entry for
"AlternatingDriver" to the service manifest.

Close apache/calcite#185


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

Branch: refs/heads/branch-release
Commit: 62576556abd0467eba7c964a90b9dc2178807f7a
Parents: af77ec8
Author: Vladimir Sitnikov <si...@gmail.com>
Authored: Mon Jan 18 15:23:32 2016 +0300
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Jan 18 11:02:02 2016 -0800

----------------------------------------------------------------------
 .../apache/calcite/avatica/remote/AlternatingRemoteMetaTest.java | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/62576556/avatica-server/src/test/java/org/apache/calcite/avatica/remote/AlternatingRemoteMetaTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/remote/AlternatingRemoteMetaTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/remote/AlternatingRemoteMetaTest.java
index ffd6f7c..6f4c51e 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/remote/AlternatingRemoteMetaTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/remote/AlternatingRemoteMetaTest.java
@@ -68,6 +68,10 @@ public class AlternatingRemoteMetaTest {
 
   static {
     try {
+      // Force DriverManager initialization before we hit AlternatingDriver->Driver.<clinit>
+      // Otherwise Driver.<clinit> -> DriverManager.registerDriver -> scan service provider files
+      // causes a deadlock; see [CALCITE-1060]
+      DriverManager.getDrivers();
       DriverManager.registerDriver(new AlternatingDriver());
     } catch (SQLException e) {
       throw new RuntimeException(e);


[37/50] [abbrv] calcite git commit: [CALCITE-975] Allow Planner to return validated row type together with SqlNode

Posted by jh...@apache.org.
[CALCITE-975] Allow Planner to return validated row type together with SqlNode

close apache/calcite#184


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

Branch: refs/heads/branch-release
Commit: 0045e01f6178df5bcc8caf780040f3cff159bb20
Parents: a67b4a9
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Tue Apr 7 03:34:06 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 12 10:22:09 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/prepare/PlannerImpl.java     |  9 +++++++++
 core/src/main/java/org/apache/calcite/tools/Planner.java | 11 +++++++++++
 2 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/0045e01f/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index d75d9c9..18d9746 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -44,6 +44,7 @@ import org.apache.calcite.tools.Planner;
 import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -180,6 +181,14 @@ public class PlannerImpl implements Planner {
     return validatedSqlNode;
   }
 
+  public Pair<SqlNode, RelDataType> validateAndGetType(SqlNode sqlNode)
+      throws ValidationException {
+    final SqlNode validatedNode = this.validate(sqlNode);
+    final RelDataType type =
+        this.validator.getValidatedNodeType(validatedNode);
+    return Pair.of(validatedNode, type);
+  }
+
   public final RelNode convert(SqlNode sql) throws RelConversionException {
     return rel(sql).rel;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/0045e01f/core/src/main/java/org/apache/calcite/tools/Planner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Planner.java b/core/src/main/java/org/apache/calcite/tools/Planner.java
index 73c8047..8b2b25d 100644
--- a/core/src/main/java/org/apache/calcite/tools/Planner.java
+++ b/core/src/main/java/org/apache/calcite/tools/Planner.java
@@ -19,9 +19,11 @@ package org.apache.calcite.tools;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.util.Pair;
 
 /**
  * A fa&ccedil;ade that covers Calcite's query planning process: parse SQL,
@@ -53,6 +55,15 @@ public interface Planner {
   SqlNode validate(SqlNode sqlNode) throws ValidationException;
 
   /**
+   * Validates a SQL statement.
+   *
+   * @param sqlNode Root node of the SQL parse tree.
+   * @return Validated node and its validated type.
+   * @throws ValidationException if not valid
+   */
+  Pair<SqlNode, RelDataType> validateAndGetType(SqlNode sqlNode) throws ValidationException;
+
+  /**
    * Converts a SQL parse tree into a tree of relational expressions.
    *
    * <p>You must call {@link #validate(org.apache.calcite.sql.SqlNode)} first.


[07/50] [abbrv] calcite git commit: Fix "endPosTable already set" from javac

Posted by jh...@apache.org.
Fix "endPosTable already set" from javac

Non clean build fails with javac exception.
It is believed that the cause is that Maven tries to run annotation pre-proessor over already pre-processed files.
Running annotation again is not allowed, thus javac failure.

The workaround is to clean ubenchmark/target directory on validate to avoid incremental compilations

More info here: https://bugs.openjdk.java.net/browse/JDK-8067747?focusedCommentId=13872639&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13872639

Exception:
An exception has occurred in the compiler (1.8.0_66). Please file a bug at the Java Bug Database (http://bugreport.java.com/bugreport/) after checking the database for duplicates. Include your program and the following diagnostic in your report.  Thank you.
java.lang.IllegalStateException: endPosTable already set
	at com.sun.tools.javac.util.DiagnosticSource.setEndPosTable(DiagnosticSource.java:136)
	at com.sun.tools.javac.util.Log.setEndPosTable(Log.java:350)
	at com.sun.tools.javac.main.JavaCompiler.parse(JavaCompiler.java:667)
	at com.sun.tools.javac.main.JavaCompiler.parseFiles(JavaCompiler.java:950)
	at com.sun.tools.javac.processing.JavacProcessingEnvironment$Round.<init>(JavacProcessingEnvironment.java:892)
	at com.sun.tools.javac.processing.JavacProcessingEnvironment$Round.next(JavacProcessingEnvironment.java:921)
	at com.sun.tools.javac.processing.JavacProcessingEnvironment.doProcessing(JavacProcessingEnvironment.java:1187)
	at com.sun.tools.javac.main.JavaCompiler.processAnnotations(JavaCompiler.java:1170)
	at com.sun.tools.javac.main.JavaCompiler.compile(JavaCompiler.java:856)
	at com.sun.tools.javac.main.Main.compile(Main.java:523)
	at com.sun.tools.javac.api.JavacTaskImpl.doCall(JavacTaskImpl.java:129)
	at com.sun.tools.javac.api.JavacTaskImpl.call(JavacTaskImpl.java:138)
	at org.codehaus.plexus.compiler.javac.JavaxToolsCompiler.compileInProcess(JavaxToolsCompiler.java:125)
	at org.codehaus.plexus.compiler.javac.JavacCompiler.performCompile(JavacCompiler.java:169)
	at org.apache.maven.plugin.compiler.AbstractCompilerMojo.execute(AbstractCompilerMojo.java:823)
	at org.apache.maven.plugin.compiler.CompilerMojo.execute(CompilerMojo.java:129)
	at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:134)
	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:208)
	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)
	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)
	at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:116)
	at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:80)
	at org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build(SingleThreadedBuilder.java:51)
	at org.apache.maven.lifecycle.internal.LifecycleStarter.execute(LifecycleStarter.java:128)
	at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:307)
	at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:193)
	at org.apache.maven.DefaultMaven.execute(DefaultMaven.java:106)
	at org.apache.maven.cli.MavenCli.execute(MavenCli.java:862)
	at org.apache.maven.cli.MavenCli.doMain(MavenCli.java:286)
	at org.apache.maven.cli.MavenCli.main(MavenCli.java:197)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:497)
	at org.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced(Launcher.java:289)
	at org.codehaus.plexus.classworlds.launcher.Launcher.launch(Launcher.java:229)
	at org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode(Launcher.java:415)
	at org.codehaus.plexus.classworlds.launcher.Launcher.main(Launcher.java:356)


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

Branch: refs/heads/branch-release
Commit: ba1eee121c52ab0c2891fd4faa423e0a55f5b5f9
Parents: 4a29b3c
Author: Vladimir Sitnikov <si...@gmail.com>
Authored: Sun Jan 10 11:23:50 2016 +0300
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:46:49 2016 -0800

----------------------------------------------------------------------
 ubenchmark/pom.xml | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/ba1eee12/ubenchmark/pom.xml
----------------------------------------------------------------------
diff --git a/ubenchmark/pom.xml b/ubenchmark/pom.xml
index e50d4dd..83e29bb 100644
--- a/ubenchmark/pom.xml
+++ b/ubenchmark/pom.xml
@@ -64,6 +64,22 @@ limitations under the License.
   <build>
     <plugins>
       <plugin>
+        <!-- This workarounds Maven issue that tries to pass annotation-pre-processed sources
+             when using incremental compilation.
+             More details here: https://bugs.openjdk.java.net/browse/JDK-8067747?focusedCommentId=13872639&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13872639
+             -->
+        <artifactId>maven-clean-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>clean-on-validate</id>
+            <goals>
+              <goal>clean</goal>
+            </goals>
+            <phase>validate</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <artifactId>maven-compiler-plugin</artifactId>
         <configuration>
           <source>1.6</source>


[39/50] [abbrv] calcite git commit: [CALCITE-977] Make the constant expression Executor configurable in FrameworkConfig (Jason Altekruse)

Posted by jh...@apache.org.
[CALCITE-977] Make the constant expression Executor configurable in FrameworkConfig (Jason Altekruse)

Close apache/calcite#182


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

Branch: refs/heads/branch-release
Commit: 5323d8d48baa2d7bc8dea8b03bc0bda93563e0f9
Parents: e15f89e
Author: Jason Altekruse <al...@gmail.com>
Authored: Fri Apr 10 13:07:35 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 12 13:47:17 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptPlanner.java   |  1 +
 .../org/apache/calcite/prepare/PlannerImpl.java  |  3 +++
 .../apache/calcite/tools/FrameworkConfig.java    |  6 ++++++
 .../org/apache/calcite/tools/Frameworks.java     | 19 +++++++++++++++++--
 4 files changed, 27 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/5323d8d4/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java b/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
index 9b5e22a..bad9402 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
@@ -318,6 +318,7 @@ public interface RelOptPlanner {
   /** Sets the object that can execute scalar expressions. */
   void setExecutor(Executor executor);
 
+  /** Returns the executor used to evaluate constant expressions. */
   Executor getExecutor();
 
   /** Called when a relational expression is copied to a similar expression. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/5323d8d4/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index 18d9746..fa6ea30 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -72,6 +72,7 @@ public class PlannerImpl implements Planner {
   private SchemaPlus defaultSchema;
   private JavaTypeFactory typeFactory;
   private RelOptPlanner planner;
+  private RelOptPlanner.Executor executor;
 
   // set in STATE_4_VALIDATE
   private CalciteSqlValidator validator;
@@ -91,6 +92,7 @@ public class PlannerImpl implements Planner {
     this.state = State.STATE_0_CLOSED;
     this.traitDefs = config.getTraitDefs();
     this.convertletTable = config.getConvertletTable();
+    this.executor = config.getExecutor();
     reset();
   }
 
@@ -135,6 +137,7 @@ public class PlannerImpl implements Planner {
             Util.discard(rootSchema); // use our own defaultSchema
             typeFactory = (JavaTypeFactory) cluster.getTypeFactory();
             planner = cluster.getPlanner();
+            planner.setExecutor(executor);
             return null;
           }
         },

http://git-wip-us.apache.org/repos/asf/calcite/blob/5323d8d4/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java b/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
index 6301321..319c216 100644
--- a/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
+++ b/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
@@ -18,6 +18,7 @@ package org.apache.calcite.tools;
 
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.schema.SchemaPlus;
@@ -46,6 +47,11 @@ public interface FrameworkConfig {
   SchemaPlus getDefaultSchema();
 
   /**
+   * Returns the executor used to evaluate constant expressions.
+   */
+  RelOptPlanner.Executor getExecutor();
+
+  /**
    * Returns a list of one or more programs used during the course of query
    * evaluation.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/5323d8d4/core/src/main/java/org/apache/calcite/tools/Frameworks.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Frameworks.java b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
index dffac61..a4e43fc 100644
--- a/core/src/main/java/org/apache/calcite/tools/Frameworks.java
+++ b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
@@ -21,6 +21,7 @@ import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptSchema;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.prepare.CalcitePrepareImpl;
@@ -179,6 +180,7 @@ public class Frameworks {
     private SqlParser.Config parserConfig =
         SqlParser.Config.DEFAULT;
     private SchemaPlus defaultSchema;
+    private RelOptPlanner.Executor executor;
     private RelOptCostFactory costFactory;
     private RelDataTypeSystem typeSystem = RelDataTypeSystem.DEFAULT;
 
@@ -187,7 +189,7 @@ public class Frameworks {
     public FrameworkConfig build() {
       return new StdFrameworkConfig(context, convertletTable, operatorTable,
           programs, traitDefs, parserConfig, defaultSchema, costFactory,
-          typeSystem);
+          typeSystem, executor);
     }
 
     public ConfigBuilder context(Context c) {
@@ -195,6 +197,12 @@ public class Frameworks {
       return this;
     }
 
+    public ConfigBuilder executor(RelOptPlanner.Executor executor) {
+      Preconditions.checkNotNull(executor);
+      this.executor = executor;
+      return this;
+    }
+
     public ConfigBuilder convertletTable(
         SqlRexConvertletTable convertletTable) {
       this.convertletTable = Preconditions.checkNotNull(convertletTable);
@@ -273,6 +281,7 @@ public class Frameworks {
     private final SchemaPlus defaultSchema;
     private final RelOptCostFactory costFactory;
     private final RelDataTypeSystem typeSystem;
+    private final RelOptPlanner.Executor executor;
 
     public StdFrameworkConfig(Context context,
         SqlRexConvertletTable convertletTable,
@@ -282,7 +291,8 @@ public class Frameworks {
         SqlParser.Config parserConfig,
         SchemaPlus defaultSchema,
         RelOptCostFactory costFactory,
-        RelDataTypeSystem typeSystem) {
+        RelDataTypeSystem typeSystem,
+        RelOptPlanner.Executor executor) {
       this.context = context;
       this.convertletTable = convertletTable;
       this.operatorTable = operatorTable;
@@ -292,6 +302,7 @@ public class Frameworks {
       this.defaultSchema = defaultSchema;
       this.costFactory = costFactory;
       this.typeSystem = typeSystem;
+      this.executor = executor;
     }
 
     public SqlParser.Config getParserConfig() {
@@ -302,6 +313,10 @@ public class Frameworks {
       return defaultSchema;
     }
 
+    public RelOptPlanner.Executor getExecutor() {
+      return executor;
+    }
+
     public ImmutableList<Program> getPrograms() {
       return programs;
     }


[02/50] [abbrv] calcite git commit: Fix RelToSqlConverterTest on Windows

Posted by jh...@apache.org.
Fix RelToSqlConverterTest on Windows


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

Branch: refs/heads/branch-release
Commit: 894dd9b522228f43d26eba8fabcf244a33029d19
Parents: d4bbf58
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jan 5 15:51:45 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/894dd9b5/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 1d26eb8..8b19898 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
@@ -27,6 +27,7 @@ import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
 import org.apache.calcite.tools.Program;
+import org.apache.calcite.util.Util;
 
 import org.junit.Test;
 
@@ -51,7 +52,7 @@ public class RelToSqlConverterTest {
       final RelToSqlConverter converter =
           new RelToSqlConverter(dialect);
       final SqlNode sqlNode = converter.visitChild(0, rel).asQuery();
-      assertThat(sqlNode.toSqlString(dialect).getSql(),
+      assertThat(Util.toLinux(sqlNode.toSqlString(dialect).getSql()),
           is(expectedQuery));
     } catch (Exception e) {
       assertTrue("Parsing failed throwing error: " + e.getMessage(), false);


[32/50] [abbrv] calcite git commit: [CALCITE-794] Detect cycles when computing statistics

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Calc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Calc.java b/core/src/main/java/org/apache/calcite/rel/core/Calc.java
index f4067eb..5fe5fcb 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Calc.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Calc.java
@@ -25,7 +25,7 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
-import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
@@ -131,17 +131,15 @@ public abstract class Calc extends SingleRel {
     return program;
   }
 
-  public double getRows() {
-    return LogicalFilter.estimateFilteredRows(
-        getInput(),
-        program);
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
+    return RelMdUtil.estimateFilteredRows(getInput(), program, mq);
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double dRows = RelMetadataQuery.getRowCount(this);
-    double dCpu =
-        RelMetadataQuery.getRowCount(getInput())
-            * program.getExprCount();
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double dRows = mq.getRowCount(this);
+    double dCpu = mq.getRowCount(getInput())
+        * program.getExprCount();
     double dIo = 0;
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Correlate.java b/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
index 982a762..f985099 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
@@ -185,19 +185,20 @@ public abstract class Correlate extends BiRel {
     return ImmutableSet.of(correlationId);
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double rowCount = RelMetadataQuery.getRowCount(this);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double rowCount = mq.getRowCount(this);
 
-    final double rightRowCount = right.getRows();
-    final double leftRowCount = left.getRows();
+    final double rightRowCount = right.estimateRowCount(mq);
+    final double leftRowCount = left.estimateRowCount(mq);
     if (Double.isInfinite(leftRowCount) || Double.isInfinite(rightRowCount)) {
       return planner.getCostFactory().makeInfiniteCost();
     }
 
-    Double restartCount = RelMetadataQuery.getRowCount(getLeft());
+    Double restartCount = mq.getRowCount(getLeft());
     // RelMetadataQuery.getCumulativeCost(getRight()); does not work for
     // RelSubset, so we ask planner to cost-estimate right relation
-    RelOptCost rightCost = planner.getCost(getRight());
+    RelOptCost rightCost = planner.getCost(getRight(), mq);
     RelOptCost rescanCost =
         rightCost.multiplyBy(Math.max(1.0, restartCount - 1));
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Exchange.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Exchange.java b/core/src/main/java/org/apache/calcite/rel/core/Exchange.java
index a9acc86..060fa1c 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Exchange.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Exchange.java
@@ -89,10 +89,11 @@ public abstract class Exchange extends SingleRel {
     return distribution;
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // Higher cost if rows are wider discourages pushing a project through an
     // exchange.
-    double rowCount = RelMetadataQuery.getRowCount(this);
+    double rowCount = mq.getRowCount(this);
     double bytesPerRow = getRowType().getFieldCount() * 4;
     return planner.getCostFactory().makeCost(
         Util.nLogN(rowCount) * bytesPerRow, rowCount, 0);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Filter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Filter.java b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
index e5461e8..5387804 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Filter.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
@@ -25,9 +25,9 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexChecker;
-import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
@@ -123,37 +123,28 @@ public abstract class Filter extends SingleRel {
     return litmus.succeed();
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double dRows = RelMetadataQuery.getRowCount(this);
-    double dCpu = RelMetadataQuery.getRowCount(getInput());
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double dRows = mq.getRowCount(this);
+    double dCpu = mq.getRowCount(getInput());
     double dIo = 0;
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
   }
 
-  // override RelNode
-  public double getRows() {
-    return estimateFilteredRows(
-        getInput(),
-        condition);
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
+    return RelMdUtil.estimateFilteredRows(getInput(), condition, mq);
   }
 
+  @Deprecated // to be removed before 2.0
   public static double estimateFilteredRows(RelNode child, RexProgram program) {
-    // convert the program's RexLocalRef condition to an expanded RexNode
-    RexLocalRef programCondition = program.getCondition();
-    RexNode condition;
-    if (programCondition == null) {
-      condition = null;
-    } else {
-      condition = program.expandLocalRef(programCondition);
-    }
-    return estimateFilteredRows(
-        child,
-        condition);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return RelMdUtil.estimateFilteredRows(child, program, mq);
   }
 
+  @Deprecated // to be removed before 2.0
   public static double estimateFilteredRows(RelNode child, RexNode condition) {
-    return RelMetadataQuery.getRowCount(child)
-        * RelMetadataQuery.getSelectivity(child, condition);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return RelMdUtil.estimateFilteredRows(child, condition, mq);
   }
 
   public RelWriter explainTerms(RelWriter pw) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Intersect.java b/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
index 6aecbf8..7495a8f 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Intersect.java
@@ -51,12 +51,11 @@ public abstract class Intersect extends SetOp {
     super(input);
   }
 
-  @Override public double getRows() {
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
     // REVIEW jvs 30-May-2005:  I just pulled this out of a hat.
     double dRows = Double.MAX_VALUE;
     for (RelNode input : inputs) {
-      dRows = Math.min(
-          dRows, RelMetadataQuery.getRowCount(input));
+      dRows = Math.min(dRows, mq.getRowCount(input));
     }
     dRows *= 0.25;
     return dRows;

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Join.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Join.java b/core/src/main/java/org/apache/calcite/rel/core/Join.java
index 3ac32d2..7c20f14 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Join.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Join.java
@@ -179,22 +179,24 @@ public abstract class Join extends BiRel {
     return litmus.succeed();
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // REVIEW jvs 9-Apr-2006:  Just for now...
-    double rowCount = RelMetadataQuery.getRowCount(this);
+    double rowCount = mq.getRowCount(this);
     return planner.getCostFactory().makeCost(rowCount, 0, 0);
   }
 
-  /** @deprecated Use {@link RelMdUtil#getJoinRowCount(Join, RexNode)}. */
+  /** @deprecated Use {@link RelMdUtil#getJoinRowCount(RelMetadataQuery, Join, RexNode)}. */
   @Deprecated // to be removed before 2.0
   public static double estimateJoinedRows(
       Join joinRel,
       RexNode condition) {
-    return Util.first(RelMdUtil.getJoinRowCount(joinRel, condition), 1D);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return Util.first(RelMdUtil.getJoinRowCount(mq, joinRel, condition), 1D);
   }
 
-  @Override public double getRows() {
-    return Util.first(RelMdUtil.getJoinRowCount(this, condition), 1D);
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
+    return Util.first(RelMdUtil.getJoinRowCount(mq, this, condition), 1D);
   }
 
   @Override public Set<CorrelationId> getVariablesSet() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Minus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Minus.java b/core/src/main/java/org/apache/calcite/rel/core/Minus.java
index 18c6f60..9950d78 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Minus.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Minus.java
@@ -21,6 +21,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.sql.SqlKind;
 
 import java.util.List;
@@ -48,8 +49,8 @@ public abstract class Minus extends SetOp {
     super(input);
   }
 
-  @Override public double getRows() {
-    return RelMdUtil.getMinusRowCount(this);
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
+    return RelMdUtil.getMinusRowCount(mq, this);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Project.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Project.java b/core/src/main/java/org/apache/calcite/rel/core/Project.java
index 86f4147..e92b04d 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Project.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Project.java
@@ -210,8 +210,9 @@ public abstract class Project extends SingleRel {
     return litmus.succeed();
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double dRows = RelMetadataQuery.getRowCount(getInput());
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double dRows = mq.getRowCount(getInput());
     double dCpu = dRows * exps.size();
     double dIo = 0;
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
index 6db45f1..571e8f6 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
@@ -23,6 +23,7 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
@@ -94,15 +95,16 @@ public class SemiJoin extends EquiJoin {
         joinInfo.leftKeys, joinInfo.rightKeys);
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // REVIEW jvs 9-Apr-2006:  Just for now...
     return planner.getCostFactory().makeTinyCost();
   }
 
-  @Override public double getRows() {
-    return Util.first(RelMdUtil.getSemiJoinRowCount(left, right, joinType, condition),
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
+    return Util.first(
+        RelMdUtil.getSemiJoinRowCount(mq, left, right, joinType, condition),
         1D);
-
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Sort.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Sort.java b/core/src/main/java/org/apache/calcite/rel/core/Sort.java
index 022479b..ec42fbf 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Sort.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Sort.java
@@ -129,10 +129,11 @@ public abstract class Sort extends SingleRel {
   public abstract Sort copy(RelTraitSet traitSet, RelNode newInput,
       RelCollation newCollation, RexNode offset, RexNode fetch);
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // Higher cost if rows are wider discourages pushing a project through a
     // sort.
-    double rowCount = RelMetadataQuery.getRowCount(this);
+    double rowCount = mq.getRowCount(this);
     double bytesPerRow = getRowType().getFieldCount() * 4;
     return planner.getCostFactory().makeCost(
         Util.nLogN(rowCount) * bytesPerRow, rowCount, 0);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java b/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
index bffa50f..7482c39 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
@@ -146,24 +146,24 @@ public abstract class TableFunctionScan extends AbstractRelNode {
   }
 
   @Override public void replaceInput(int ordinalInParent, RelNode p) {
-    final List<RelNode> newInputs = new ArrayList<RelNode>(inputs);
+    final List<RelNode> newInputs = new ArrayList<>(inputs);
     newInputs.set(ordinalInParent, p);
     inputs = ImmutableList.copyOf(newInputs);
     recomputeDigest();
   }
 
-  @Override public double getRows() {
-    // Calculate result as the sum of the input rowcount estimates,
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
+    // Calculate result as the sum of the input row count estimates,
     // assuming there are any, otherwise use the superclass default.  So
     // for a no-input UDX, behave like an AbstractRelNode; for a one-input
     // UDX, behave like a SingleRel; for a multi-input UDX, behave like
     // UNION ALL.  TODO jvs 10-Sep-2007: UDX-supplied costing metadata.
     if (inputs.size() == 0) {
-      return super.getRows();
+      return super.estimateRowCount(mq);
     }
     double nRows = 0.0;
     for (RelNode input : inputs) {
-      Double d = RelMetadataQuery.getRowCount(input);
+      Double d = mq.getRowCount(input);
       if (d != null) {
         nRows += d;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableModify.java b/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
index fea3a5e..79b6a0d 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
@@ -54,7 +54,7 @@ public abstract class TableModify extends SingleRel {
    * Enumeration of supported modification operations.
    */
   public enum Operation {
-    INSERT, UPDATE, DELETE, MERGE;
+    INSERT, UPDATE, DELETE, MERGE
   }
 
   //~ Instance fields --------------------------------------------------------
@@ -190,10 +190,10 @@ public abstract class TableModify extends SingleRel {
         .item("flattened", flattened);
   }
 
-  // implement RelNode
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // REVIEW jvs 21-Apr-2006:  Just for now...
-    double rowCount = RelMetadataQuery.getRowCount(this);
+    double rowCount = mq.getRowCount(this);
     return planner.getCostFactory().makeCost(rowCount, 0, 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableScan.java b/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
index 2b958ef..27f842d 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
@@ -70,7 +71,7 @@ public abstract class TableScan extends AbstractRelNode {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public double getRows() {
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
     return table.getRowCount();
   }
 
@@ -82,7 +83,8 @@ public abstract class TableScan extends AbstractRelNode {
     return table.getCollationList();
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     double dRows = table.getRowCount();
     double dCpu = dRows + 1; // ensure non-zero cost
     double dIo = 0;

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Union.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Union.java b/core/src/main/java/org/apache/calcite/rel/core/Union.java
index 1907199..f7c6a42 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Union.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Union.java
@@ -20,6 +20,7 @@ import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.sql.SqlKind;
 
@@ -51,27 +52,19 @@ public abstract class Union extends SetOp {
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelNode
-  public double getRows() {
-    double dRows = estimateRowCount(this);
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
+    double dRows = RelMdUtil.getUnionAllRowCount(RelMetadataQuery.instance(),
+        this);
     if (!all) {
       dRows *= 0.5;
     }
     return dRows;
   }
 
-  /**
-   * Helper method for computing row count for UNION ALL.
-   *
-   * @param rel node representing UNION ALL
-   * @return estimated row count for rel
-   */
+  @Deprecated // to be removed before 2.0
   public static double estimateRowCount(RelNode rel) {
-    double dRows = 0;
-    for (RelNode input : rel.getInputs()) {
-      dRows += RelMetadataQuery.getRowCount(input);
-    }
-    return dRows;
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return RelMdUtil.getUnionAllRowCount(mq, (Union) rel);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Values.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Values.java b/core/src/main/java/org/apache/calcite/rel/core/Values.java
index 2f4b6ba..bf99fae 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Values.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Values.java
@@ -73,7 +73,7 @@ public abstract class Values extends AbstractRelNode {
 
   //~ Instance fields --------------------------------------------------------
 
-  protected final ImmutableList<ImmutableList<RexLiteral>> tuples;
+  public final ImmutableList<ImmutableList<RexLiteral>> tuples;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -143,14 +143,13 @@ public abstract class Values extends AbstractRelNode {
     return true;
   }
 
-  // implement RelNode
-  protected RelDataType deriveRowType() {
+  @Override protected RelDataType deriveRowType() {
     return rowType;
   }
 
-  // implement RelNode
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double dRows = RelMetadataQuery.getRowCount(this);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double dRows = mq.getRowCount(this);
 
     // Assume CPU is negligible since values are precomputed.
     double dCpu = 1;
@@ -159,7 +158,7 @@ public abstract class Values extends AbstractRelNode {
   }
 
   // implement RelNode
-  public double getRows() {
+  public double estimateRowCount(RelMetadataQuery mq) {
     return tuples.size();
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/core/Window.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Window.java b/core/src/main/java/org/apache/calcite/rel/core/Window.java
index 7efb115..09d9ee7 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Window.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Window.java
@@ -170,14 +170,15 @@ public abstract class Window extends SingleRel {
     return constants;
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // Cost is proportional to the number of rows and the number of
     // components (groups and aggregate functions). There is
     // no I/O cost.
     //
     // TODO #1. Add memory cost.
     // TODO #2. MIN and MAX have higher CPU cost than SUM and COUNT.
-    final double rowsIn = RelMetadataQuery.getRowCount(getInput());
+    final double rowsIn = mq.getRowCount(getInput());
     int count = groups.size();
     for (Group group : groups) {
       count += group.aggCalls.size();

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java
index 63fb3ab..9386f81 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelWriterImpl.java
@@ -40,8 +40,8 @@ public class RelWriterImpl implements RelWriter {
   private final SqlExplainLevel detailLevel;
   private final boolean withIdPrefix;
   protected final Spacer spacer = new Spacer();
-  private final List<Pair<String, Object>> values =
-      new ArrayList<Pair<String, Object>>();
+  private final List<Pair<String, Object>> values = new ArrayList<>();
+  protected final RelMetadataQuery mq = RelMetadataQuery.instance();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -59,14 +59,11 @@ public class RelWriterImpl implements RelWriter {
 
   //~ Methods ----------------------------------------------------------------
 
-  protected void explain_(
-      RelNode rel,
+  protected void explain_(RelNode rel,
       List<Pair<String, Object>> values) {
     List<RelNode> inputs = rel.getInputs();
 
-    if (!RelMetadataQuery.isVisibleInExplain(
-        rel,
-        detailLevel)) {
+    if (!mq.isVisibleInExplain(rel, detailLevel)) {
       // render children in place of this, at same level
       explainInputs(inputs);
       return;
@@ -101,9 +98,9 @@ public class RelWriterImpl implements RelWriter {
     switch (detailLevel) {
     case ALL_ATTRIBUTES:
       s.append(": rowcount = ")
-          .append(RelMetadataQuery.getRowCount(rel))
+          .append(mq.getRowCount(rel))
           .append(", cumulative cost = ")
-          .append(RelMetadataQuery.getCumulativeCost(rel));
+          .append(mq.getCumulativeCost(rel));
     }
     switch (detailLevel) {
     case NON_COST_ATTRIBUTES:

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
index 3ca1645..3341ebf 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
@@ -29,6 +29,7 @@ import org.apache.calcite.rel.core.Calc;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rules.FilterToCalcRule;
 import org.apache.calcite.rel.rules.ProjectToCalcRule;
 import org.apache.calcite.rex.RexNode;
@@ -90,18 +91,19 @@ public final class LogicalCalc extends Calc {
   public static LogicalCalc create(final RelNode input,
       final RexProgram program) {
     final RelOptCluster cluster = input.getCluster();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet = cluster.traitSet()
         .replace(Convention.NONE)
         .replaceIfs(RelCollationTraitDef.INSTANCE,
             new Supplier<List<RelCollation>>() {
               public List<RelCollation> get() {
-                return RelMdCollation.calc(input, program);
+                return RelMdCollation.calc(mq, input, program);
               }
             })
         .replaceIf(RelDistributionTraitDef.INSTANCE,
             new Supplier<RelDistribution>() {
               public RelDistribution get() {
-                return RelMdDistribution.calc(input, program);
+                return RelMdDistribution.calc(mq, input, program);
               }
             });
     return new LogicalCalc(cluster, traitSet, input, program);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
index d25874e..1c25fba 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 
 import com.google.common.base.Preconditions;
@@ -106,17 +107,18 @@ public final class LogicalFilter extends Filter {
   public static LogicalFilter create(final RelNode input, RexNode condition,
       ImmutableSet<CorrelationId> variablesSet) {
     final RelOptCluster cluster = input.getCluster();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE)
         .replaceIfs(RelCollationTraitDef.INSTANCE,
             new Supplier<List<RelCollation>>() {
               public List<RelCollation> get() {
-                return RelMdCollation.filter(input);
+                return RelMdCollation.filter(mq, input);
               }
             })
         .replaceIf(RelDistributionTraitDef.INSTANCE,
             new Supplier<RelDistribution>() {
               public RelDistribution get() {
-                return RelMdDistribution.filter(input);
+                return RelMdDistribution.filter(mq, input);
               }
             });
     return new LogicalFilter(cluster, traitSet, input, condition, variablesSet);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
index cce5e79..61f2fd7 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.metadata.RelMdCollation;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
@@ -111,13 +112,14 @@ public final class LogicalProject extends Project {
   public static LogicalProject create(final RelNode input,
       final List<? extends RexNode> projects, RelDataType rowType) {
     final RelOptCluster cluster = input.getCluster();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet =
         cluster.traitSet().replace(Convention.NONE)
             .replaceIfs(
                 RelCollationTraitDef.INSTANCE,
                 new Supplier<List<RelCollation>>() {
                   public List<RelCollation> get() {
-                    return RelMdCollation.project(input, projects);
+                    return RelMdCollation.project(mq, input, projects);
                   }
                 });
     return new LogicalProject(cluster, traitSet, input, projects, rowType);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java
index 79102db..ec18685 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableFunctionScan.java
@@ -25,6 +25,7 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.TableFunctionScan;
 import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 
@@ -111,7 +112,7 @@ public class LogicalTableFunctionScan extends TableFunctionScan {
         columnMappings);
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
     // REVIEW jvs 8-Jan-2006:  what is supposed to be here
     // for an abstract rel?
     return planner.getCostFactory().makeHugeCost();

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
index 889f7e6..6d84d74 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
@@ -26,6 +26,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rel.metadata.RelMdCollation;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -81,11 +82,12 @@ public class LogicalValues extends Values {
   public static LogicalValues create(RelOptCluster cluster,
       final RelDataType rowType,
       final ImmutableList<ImmutableList<RexLiteral>> tuples) {
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE)
         .replaceIfs(
             RelCollationTraitDef.INSTANCE, new Supplier<List<RelCollation>>() {
               public List<RelCollation> get() {
-                return RelMdCollation.values(rowType, tuples);
+                return RelMdCollation.values(mq, rowType, tuples);
               }
             });
     return new LogicalValues(cluster, traitSet, rowType, tuples);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java b/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
index 81d6eed..87fbbf4 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java
@@ -122,7 +122,7 @@ public abstract class BuiltInMetadata {
     /**
      * Estimates the number of rows which will be returned by a relational
      * expression. The default implementation for this query asks the rel itself
-     * via {@link RelNode#getRows}, but metadata providers can override this
+     * via {@link RelNode#estimateRowCount}, but metadata providers can override this
      * with their own cost models.
      *
      * @return estimated row count, or null if no reliable estimate can be
@@ -131,14 +131,18 @@ public abstract class BuiltInMetadata {
     Double getRowCount();
   }
 
-  /** Metadata about the max number of rows returned by a relational expression. */
+  /** Metadata about the maximum number of rows returned by a relational
+   * expression. */
   public interface MaxRowCount extends Metadata {
     /**
      * Estimates the max number of rows which will be returned by a relational
-     * expression. The default implementation for this query returns Double.POSITIVE_INFINITY,
+     * expression.
+     *
+     * <p>The default implementation for this query returns
+     * {@link Double#POSITIVE_INFINITY},
      * but metadata providers can override this with their own cost models.
      *
-     * @return estimated max row count
+     * @return upper bound on the number of rows returned
      */
     Double getMaxRowCount();
   }
@@ -338,7 +342,7 @@ public abstract class BuiltInMetadata {
      * physical operator implementing this relational expression, and all other
      * operators within the same phase, across all splits.
      *
-     * @see org.apache.calcite.rel.metadata.BuiltInMetadata.Parallelism#splitCount()
+     * @see Parallelism#splitCount()
      */
     Double cumulativeMemoryWithinPhase();
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
index 516de3a..8438a8d 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
@@ -19,11 +19,12 @@ package org.apache.calcite.rel.metadata;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rel.RelNode;
 
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.util.HashMap;
@@ -37,18 +38,12 @@ import java.util.Map;
 public class CachingRelMetadataProvider implements RelMetadataProvider {
   //~ Instance fields --------------------------------------------------------
 
-  private final Map<List, CacheEntry> cache;
+  private final Map<List, CacheEntry> cache = new HashMap<>();
 
   private final RelMetadataProvider underlyingProvider;
 
   private final RelOptPlanner planner;
 
-  private static final Object NULL_SENTINEL = new Object() {
-    @Override public String toString() {
-      return "{null}";
-    }
-  };
-
   //~ Constructors -----------------------------------------------------------
 
   public CachingRelMetadataProvider(
@@ -56,15 +51,14 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
       RelOptPlanner planner) {
     this.underlyingProvider = underlyingProvider;
     this.planner = planner;
-
-    cache = new HashMap<List, CacheEntry>();
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public Function<RelNode, Metadata> apply(Class<? extends RelNode> relClass,
-      final Class<? extends Metadata> metadataClass) {
-    final Function<RelNode, Metadata> function =
+  public <M extends Metadata> UnboundMetadata<M>
+  apply(Class<? extends RelNode> relClass,
+      final Class<? extends M> metadataClass) {
+    final UnboundMetadata<M> function =
         underlyingProvider.apply(relClass, metadataClass);
     if (function == null) {
       return null;
@@ -72,11 +66,13 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
 
     // TODO jvs 30-Mar-2006: Use meta-metadata to decide which metadata
     // query results can stay fresh until the next Ice Age.
-    return new Function<RelNode, Metadata>() {
-      public Metadata apply(RelNode input) {
-        final Metadata metadata = function.apply(input);
-        return (Metadata) Proxy.newProxyInstance(metadataClass.getClassLoader(),
-            new Class[]{metadataClass}, new CachingInvocationHandler(metadata));
+    return new UnboundMetadata<M>() {
+      public M bind(RelNode rel, RelMetadataQuery mq) {
+        final Metadata metadata = function.bind(rel, mq);
+        return metadataClass.cast(
+            Proxy.newProxyInstance(metadataClass.getClassLoader(),
+                new Class[]{metadataClass},
+                new CachingInvocationHandler(metadata)));
       }
     };
   }
@@ -113,7 +109,7 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
       if (args != null) {
         for (Object arg : args) {
           // Replace null values because ImmutableList does not allow them.
-          builder.add(arg == null ? NULL_SENTINEL : arg);
+          builder.add(NullSentinel.mask(arg));
         }
       }
       List<Object> key = builder.build();
@@ -129,14 +125,19 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
       }
 
       // Cache miss or stale.
-      Object result = method.invoke(metadata, args);
-      if (result != null) {
-        entry = new CacheEntry();
-        entry.timestamp = timestamp;
-        entry.result = result;
-        cache.put(key, entry);
+      try {
+        Object result = method.invoke(metadata, args);
+        if (result != null) {
+          entry = new CacheEntry();
+          entry.timestamp = timestamp;
+          entry.result = result;
+          cache.put(key, entry);
+        }
+        return result;
+      } catch (InvocationTargetException e) {
+        Throwables.propagateIfPossible(e.getCause());
+        throw e;
       }
-      return result;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
index 445cce4..eb8aec5 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
@@ -18,13 +18,15 @@ package org.apache.calcite.rel.metadata;
 
 import org.apache.calcite.rel.RelNode;
 
-import com.google.common.base.Function;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
+import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -53,11 +55,12 @@ public class ChainedRelMetadataProvider implements RelMetadataProvider {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Function<RelNode, Metadata> apply(Class<? extends RelNode> relClass,
-      final Class<? extends Metadata> metadataClass) {
-    final List<Function<RelNode, Metadata>> functions = Lists.newArrayList();
+  public <M extends Metadata> UnboundMetadata<M>
+  apply(Class<? extends RelNode> relClass,
+      final Class<? extends M> metadataClass) {
+    final List<UnboundMetadata<M>> functions = new ArrayList<>();
     for (RelMetadataProvider provider : providers) {
-      final Function<RelNode, Metadata> function =
+      final UnboundMetadata<M> function =
           provider.apply(relClass, metadataClass);
       if (function == null) {
         continue;
@@ -70,22 +73,21 @@ public class ChainedRelMetadataProvider implements RelMetadataProvider {
     case 1:
       return functions.get(0);
     default:
-      return new Function<RelNode, Metadata>() {
-        public Metadata apply(RelNode input) {
+      return new UnboundMetadata<M>() {
+        public M bind(RelNode rel, RelMetadataQuery mq) {
           final List<Metadata> metadataList = Lists.newArrayList();
-          for (Function<RelNode, Metadata> function : functions) {
-            final Metadata metadata = function.apply(input);
+          for (UnboundMetadata<M> function : functions) {
+            final Metadata metadata = function.bind(rel, mq);
             if (metadata != null) {
               metadataList.add(metadata);
             }
           }
-          return (Metadata) Proxy.newProxyInstance(
-              metadataClass.getClassLoader(),
-              new Class[]{metadataClass},
-              new ChainedInvocationHandler(metadataList));
+          return metadataClass.cast(
+              Proxy.newProxyInstance(metadataClass.getClassLoader(),
+                  new Class[]{metadataClass},
+                  new ChainedInvocationHandler(metadataList)));
         }
       };
-
     }
   }
 
@@ -106,9 +108,17 @@ public class ChainedRelMetadataProvider implements RelMetadataProvider {
     public Object invoke(Object proxy, Method method, Object[] args)
         throws Throwable {
       for (Metadata metadata : metadataList) {
-        final Object o = method.invoke(metadata, args);
-        if (o != null) {
-          return o;
+        try {
+          final Object o = method.invoke(metadata, args);
+          if (o != null) {
+            return o;
+          }
+        } catch (InvocationTargetException e) {
+          if (e.getCause() instanceof CyclicMetadataException) {
+            continue;
+          }
+          Throwables.propagateIfPossible(e.getCause());
+          throw e;
         }
       }
       return null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/CyclicMetadataException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/CyclicMetadataException.java b/core/src/main/java/org/apache/calcite/rel/metadata/CyclicMetadataException.java
new file mode 100644
index 0000000..d4dd249
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/CyclicMetadataException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rel.metadata;
+
+/**
+ * Exception that indicates that a cycle has been detected while
+ * computing metadata.
+ */
+public class CyclicMetadataException extends RuntimeException {
+  /** Singleton instance. Since this exception is thrown for signaling purposes,
+   * rather than on an actual error, re-using a singleton instance saves the
+   * effort of constructing an exception instance. */
+  @SuppressWarnings("ThrowableInstanceNeverThrown")
+  public static final CyclicMetadataException INSTANCE =
+      new CyclicMetadataException();
+
+  /** Creates a CyclicMetadataException. */
+  private CyclicMetadataException() {
+    super();
+  }
+}
+
+// End CyclicMetadataException.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java b/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java
index 34e456f..d5d642d 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/Metadata.java
@@ -26,7 +26,7 @@ import org.apache.calcite.rel.RelNode;
  * kinds of metadata for particular sub-classes of {@link RelNode}.
  *
  * <p>User code (typically in a planner rule or an implementation of
- * {@link RelNode#computeSelfCost(org.apache.calcite.plan.RelOptPlanner)})
+ * {@link RelNode#computeSelfCost(org.apache.calcite.plan.RelOptPlanner, RelMetadataQuery)})
  * acquires a {@code Metadata} instance by calling {@link RelNode#metadata}.
  *
  * <p>A {@code Metadata} instance already knows which particular {@code RelNode}

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java
index 7651ecd..28ca883 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactory.java
@@ -31,8 +31,17 @@ import org.apache.calcite.rel.RelNode;
 public interface MetadataFactory {
   /** Returns a metadata interface to get a particular kind of metadata
    * from a particular relational expression. Returns null if that kind of
-   * metadata is not available. */
-  <T extends Metadata> T query(RelNode rel, Class<T> clazz);
+   * metadata is not available.
+   *
+   * @param <M> Metadata type
+   *
+   * @param rel Relational expression
+   * @param mq Metadata query
+   * @param metadataClazz Metadata class
+   * @return Metadata bound to {@code rel} and {@code query}
+   */
+  <M extends Metadata> M query(RelNode rel, RelMetadataQuery mq,
+      Class<M> metadataClazz);
 }
 
 // End MetadataFactory.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
index e20d6ce..f5b5717 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
@@ -19,8 +19,6 @@ package org.apache.calcite.rel.metadata;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.util.Pair;
 
-import com.google.common.base.Function;
-import com.google.common.base.Functions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -36,24 +34,27 @@ import java.util.concurrent.ExecutionException;
  */
 public class MetadataFactoryImpl implements MetadataFactory {
   @SuppressWarnings("unchecked")
-  public static final Function<RelNode, Metadata> DUMMY =
-      (Function) Functions.<Metadata>constant(null);
+  public static final UnboundMetadata<Metadata> DUMMY =
+      new UnboundMetadata<Metadata>() {
+        public Metadata bind(RelNode rel, RelMetadataQuery mq) {
+          return null;
+        }
+      };
 
   private final LoadingCache<
-      Pair<Class<RelNode>, Class<Metadata>>,
-      Function<RelNode, Metadata>> cache;
+      Pair<Class<RelNode>, Class<Metadata>>, UnboundMetadata<Metadata>> cache;
 
   public MetadataFactoryImpl(RelMetadataProvider provider) {
     this.cache = CacheBuilder.newBuilder().build(loader(provider));
   }
 
   static CacheLoader<Pair<Class<RelNode>, Class<Metadata>>,
-      Function<RelNode, Metadata>> loader(final RelMetadataProvider provider) {
+      UnboundMetadata<Metadata>> loader(final RelMetadataProvider provider) {
     return new CacheLoader<Pair<Class<RelNode>, Class<Metadata>>,
-        Function<RelNode, Metadata>>() {
-      @Override public Function<RelNode, Metadata> load(
+        UnboundMetadata<Metadata>>() {
+      @Override public UnboundMetadata<Metadata> load(
           Pair<Class<RelNode>, Class<Metadata>> key) throws Exception {
-        final Function<RelNode, Metadata> function =
+        final UnboundMetadata<Metadata> function =
             provider.apply(key.left, key.right);
         // Return DUMMY, not null, so the cache knows to not ask again.
         return function != null ? function : DUMMY;
@@ -61,14 +62,14 @@ public class MetadataFactoryImpl implements MetadataFactory {
     };
   }
 
-  public <T extends Metadata> T query(RelNode rel, Class<T> clazz) {
+  public <M extends Metadata> M query(RelNode rel, RelMetadataQuery mq,
+      Class<M> metadataClazz) {
     try {
       //noinspection unchecked
       final Pair<Class<RelNode>, Class<Metadata>> key =
-          (Pair) Pair.of(rel.getClass(), clazz);
-      final Metadata apply = cache.get(key).apply(rel);
-      //noinspection unchecked
-      return (T) apply;
+          (Pair) Pair.of(rel.getClass(), metadataClazz);
+      final Metadata apply = cache.get(key).bind(rel, mq);
+      return metadataClazz.cast(apply);
     } catch (ExecutionException e) {
       if (e.getCause() instanceof RuntimeException) {
         throw (RuntimeException) e.getCause();

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
index 7db2c6a..beaedfe 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
@@ -17,23 +17,29 @@
 package org.apache.calcite.rel.metadata;
 
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.ReflectiveVisitor;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Proxy;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -54,8 +60,8 @@ public class ReflectiveRelMetadataProvider
     implements RelMetadataProvider, ReflectiveVisitor {
 
   //~ Instance fields --------------------------------------------------------
-  private final Map<Class<RelNode>, Function<RelNode, Metadata>> map;
-  private final Class<?> metadataClass0;
+  private final Map<Class<RelNode>, UnboundMetadata> map;
+  private final Class<? extends Metadata> metadataClass0;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -66,8 +72,8 @@ public class ReflectiveRelMetadataProvider
    * @param metadataClass0 Metadata class
    */
   protected ReflectiveRelMetadataProvider(
-      Map<Class<RelNode>, Function<RelNode, Metadata>> map,
-      Class<?> metadataClass0) {
+      Map<Class<RelNode>, UnboundMetadata> map,
+      Class<? extends Metadata> metadataClass0) {
     assert !map.isEmpty() : "are your methods named wrong?";
     this.map = map;
     this.metadataClass0 = metadataClass0;
@@ -77,7 +83,7 @@ public class ReflectiveRelMetadataProvider
    * methods with a preceding argument.
    *
    * <p>For example, {@link BuiltInMetadata.Selectivity} has a method
-   * {@link BuiltInMetadata.Selectivity#getSelectivity(org.apache.calcite.rex.RexNode)}.
+   * {@link BuiltInMetadata.Selectivity#getSelectivity(RexNode)}.
    * A class</p>
    *
    * <blockquote><pre><code>
@@ -106,7 +112,8 @@ public class ReflectiveRelMetadataProvider
       final ImmutableList<Method> methods) {
     assert methods.size() > 0;
     final Method method0 = methods.get(0);
-    final Class<?> metadataClass0 = method0.getDeclaringClass();
+    @SuppressWarnings("unchecked")
+    final Class<Metadata> metadataClass0 = (Class) method0.getDeclaringClass();
     assert Metadata.class.isAssignableFrom(metadataClass0);
     for (Method method : methods) {
       assert method.getDeclaringClass() == metadataClass0;
@@ -128,8 +135,7 @@ public class ReflectiveRelMetadataProvider
       }
     }
 
-    final Map<Class<RelNode>, Function<RelNode, Metadata>> methodsMap =
-        Maps.newHashMap();
+    final Map<Class<RelNode>, UnboundMetadata> methodsMap = new HashMap<>();
     for (Class<RelNode> key : classes) {
       ImmutableNullableList.Builder<Method> builder =
           ImmutableNullableList.builder();
@@ -137,9 +143,10 @@ public class ReflectiveRelMetadataProvider
         builder.add(find(handlerMap, key, method));
       }
       final List<Method> handlerMethods = builder.build();
-      final Function<RelNode, Metadata> function =
-          new Function<RelNode, Metadata>() {
-            public Metadata apply(final RelNode rel) {
+      final UnboundMetadata function =
+          new UnboundMetadata() {
+            public Metadata bind(final RelNode rel,
+                final RelMetadataQuery mq) {
               return (Metadata) Proxy.newProxyInstance(
                   metadataClass0.getClassLoader(),
                   new Class[]{metadataClass0},
@@ -166,20 +173,46 @@ public class ReflectiveRelMetadataProvider
                         throw new AssertionError("not handled: " + method
                             + " for " + rel);
                       }
+                      final Method handlerMethod = handlerMethods.get(i);
+                      if (handlerMethod == null) {
+                        throw new AssertionError("not handled: " + method
+                            + " for " + rel);
+                      }
                       final Object[] args1;
+                      final List key;
                       if (args == null) {
-                        args1 = new Object[]{rel};
+                        args1 = new Object[]{rel, mq};
+                        key = FlatLists.of(rel, method);
                       } else {
-                        args1 = new Object[args.length + 1];
+                        args1 = new Object[args.length + 2];
                         args1[0] = rel;
-                        System.arraycopy(args, 0, args1, 1, args.length);
+                        args1[1] = mq;
+                        System.arraycopy(args, 0, args1, 2, args.length);
+
+                        final Object[] args2 = args1.clone();
+                        args2[1] = method; // replace RelMetadataQuery with method
+                        for (int j = 0; j < args2.length; j++) {
+                          if (args2[j] == null) {
+                            args2[j] = NullSentinel.INSTANCE;
+                          } else if (args2[j] instanceof RexNode) {
+                            // Can't use RexNode.equals - it is not deep
+                            args2[j] = args2[j].toString();
+                          }
+                        }
+                        key = FlatLists.copyOf(args2);
                       }
-                      final Method handlerMethod = handlerMethods.get(i);
-                      if (handlerMethod == null) {
-                        throw new AssertionError("not handled: " + method
-                            + " for " + rel);
+                      if (!mq.set.add(key)) {
+                        throw CyclicMetadataException.INSTANCE;
+                      }
+                      try {
+                        return handlerMethod.invoke(target, args1);
+                      } catch (InvocationTargetException
+                          | UndeclaredThrowableException e) {
+                        Throwables.propagateIfPossible(e.getCause());
+                        throw e;
+                      } finally {
+                        mq.set.remove(key);
                       }
-                      return handlerMethod.invoke(target, args1);
                     }
                   });
             }
@@ -229,19 +262,18 @@ public class ReflectiveRelMetadataProvider
     }
     final Class<?>[] parameterTypes1 = handlerMethod.getParameterTypes();
     final Class<?>[] parameterTypes = method.getParameterTypes();
-    if (parameterTypes1.length != parameterTypes.length + 1
-        || !RelNode.class.isAssignableFrom(parameterTypes1[0])) {
-      return false;
-    }
-    return Util.skip(Arrays.asList(parameterTypes1))
-        .equals(Arrays.asList(parameterTypes));
+    return parameterTypes1.length == parameterTypes.length + 2
+        && RelNode.class.isAssignableFrom(parameterTypes1[0])
+        && RelMetadataQuery.class == parameterTypes1[1]
+        && Arrays.asList(parameterTypes)
+            .equals(Util.skip(Arrays.asList(parameterTypes1), 2));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public Function<RelNode, Metadata> apply(
-      Class<? extends RelNode> relClass,
-      Class<? extends Metadata> metadataClass) {
+  public <M extends Metadata> UnboundMetadata<M>
+  apply(Class<? extends RelNode> relClass,
+      Class<? extends M> metadataClass) {
     if (metadataClass == metadataClass0) {
       return apply(relClass);
     } else {
@@ -250,11 +282,11 @@ public class ReflectiveRelMetadataProvider
   }
 
   @SuppressWarnings({ "unchecked", "SuspiciousMethodCalls" })
-  private synchronized Function<RelNode, Metadata> apply(
-      Class<? extends RelNode> relClass) {
-    List<Class<? extends RelNode>> newSources = Lists.newArrayList();
+  public <M extends Metadata> UnboundMetadata<M>
+  apply(Class<? extends RelNode> relClass) {
+    List<Class<? extends RelNode>> newSources = new ArrayList<>();
     for (;;) {
-      final Function<RelNode, Metadata> function = map.get(relClass);
+      UnboundMetadata<M> function = map.get(relClass);
       if (function != null) {
         for (@SuppressWarnings("rawtypes") Class clazz : newSources) {
           map.put(clazz, function);
@@ -265,7 +297,7 @@ public class ReflectiveRelMetadataProvider
       }
       for (Class<?> interfaceClass : relClass.getInterfaces()) {
         if (RelNode.class.isAssignableFrom(interfaceClass)) {
-          final Function<RelNode, Metadata> function2 = map.get(interfaceClass);
+          final UnboundMetadata<M> function2 = map.get(interfaceClass);
           if (function2 != null) {
             for (@SuppressWarnings("rawtypes") Class clazz : newSources) {
               map.put(clazz, function2);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
index 37ce6da..92b325d 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
@@ -47,12 +47,12 @@ import org.apache.calcite.util.ImmutableIntList;
 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.LinkedListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Ordering;
-import com.google.common.collect.Sets;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -60,6 +60,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
+import java.util.TreeSet;
 
 /**
  * RelMdCollation supplies a default implementation of
@@ -77,7 +78,9 @@ public class RelMdCollation {
 
   //~ Methods ----------------------------------------------------------------
 
-  /** Fallback method to deduce collations for any relational expression not
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.Collation#collations()},
+   * invoked using reflection, for any relational expression not
    * handled by a more specific method.
    *
    * <p>{@link org.apache.calcite.rel.core.Union},
@@ -91,60 +94,72 @@ public class RelMdCollation {
    *
    * @param rel Relational expression
    * @return Relational expression's collations
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#collations(RelNode)
    */
-  public ImmutableList<RelCollation> collations(RelNode rel) {
+  public ImmutableList<RelCollation> collations(RelNode rel,
+      RelMetadataQuery mq) {
     return ImmutableList.of();
   }
 
-  public ImmutableList<RelCollation> collations(Window rel) {
-    return ImmutableList.copyOf(window(rel.getInput(), rel.groups));
+  public ImmutableList<RelCollation> collations(Window rel,
+      RelMetadataQuery mq) {
+    return ImmutableList.copyOf(window(mq, rel.getInput(), rel.groups));
   }
 
-  public ImmutableList<RelCollation> collations(Filter rel) {
-    return RelMetadataQuery.collations(rel.getInput());
+  public ImmutableList<RelCollation> collations(Filter rel,
+      RelMetadataQuery mq) {
+    return mq.collations(rel.getInput());
   }
 
-  public ImmutableList<RelCollation> collations(TableScan scan) {
+  public ImmutableList<RelCollation> collations(TableScan scan,
+      RelMetadataQuery mq) {
     return ImmutableList.copyOf(table(scan.getTable()));
   }
 
-  public ImmutableList<RelCollation> collations(EnumerableMergeJoin join) {
+  public ImmutableList<RelCollation> collations(EnumerableMergeJoin join,
+      RelMetadataQuery mq) {
     // In general a join is not sorted. But a merge join preserves the sort
     // order of the left and right sides.
     return ImmutableList.copyOf(
-        RelMdCollation.mergeJoin(join.getLeft(),
-            join.getRight(),
-            join.getLeftKeys(),
-            join.getRightKeys()));
+        RelMdCollation.mergeJoin(mq, join.getLeft(), join.getRight(),
+            join.getLeftKeys(), join.getRightKeys()));
   }
 
-  public ImmutableList<RelCollation> collations(Sort sort) {
+  public ImmutableList<RelCollation> collations(Sort sort,
+      RelMetadataQuery mq) {
     return ImmutableList.copyOf(
         RelMdCollation.sort(sort.getCollation()));
   }
 
-  public ImmutableList<RelCollation> collations(SortExchange sort) {
+  public ImmutableList<RelCollation> collations(SortExchange sort,
+      RelMetadataQuery mq) {
     return ImmutableList.copyOf(
         RelMdCollation.sort(sort.getCollation()));
   }
 
-  public ImmutableList<RelCollation> collations(Project project) {
+  public ImmutableList<RelCollation> collations(Project project,
+      RelMetadataQuery mq) {
     return ImmutableList.copyOf(
-        project(project.getInput(), project.getProjects()));
+        project(mq, project.getInput(), project.getProjects()));
   }
 
-  public ImmutableList<RelCollation> collations(Values values) {
+  public ImmutableList<RelCollation> collations(Values values,
+      RelMetadataQuery mq) {
     return ImmutableList.copyOf(
-        values(values.getRowType(), values.getTuples()));
+        values(mq, values.getRowType(), values.getTuples()));
   }
 
-  public ImmutableList<RelCollation> collations(HepRelVertex rel) {
-    return RelMetadataQuery.collations(rel.getCurrentRel());
+  public ImmutableList<RelCollation> collations(HepRelVertex rel,
+      RelMetadataQuery mq) {
+    return mq.collations(rel.getCurrentRel());
   }
 
-  public ImmutableList<RelCollation> collations(RelSubset rel) {
+  public ImmutableList<RelCollation> collations(RelSubset rel,
+      RelMetadataQuery mq) {
     return ImmutableList.copyOf(
-        rel.getTraitSet().getTraits(RelCollationTraitDef.INSTANCE));
+        Preconditions.checkNotNull(
+            rel.getTraitSet().getTraits(RelCollationTraitDef.INSTANCE)));
   }
 
   // Helper methods
@@ -163,29 +178,28 @@ public class RelMdCollation {
 
   /** Helper method to determine a
    * {@link org.apache.calcite.rel.core.Filter}'s collation. */
-  public static List<RelCollation> filter(RelNode input) {
-    return RelMetadataQuery.collations(input);
+  public static List<RelCollation> filter(RelMetadataQuery mq, RelNode input) {
+    return mq.collations(input);
   }
 
   /** Helper method to determine a
    * limit's collation. */
-  public static List<RelCollation> limit(RelNode input) {
-    return RelMetadataQuery.collations(input);
+  public static List<RelCollation> limit(RelMetadataQuery mq, RelNode input) {
+    return mq.collations(input);
   }
 
   /** Helper method to determine a
    * {@link org.apache.calcite.rel.core.Calc}'s collation. */
-  public static List<RelCollation> calc(RelNode input,
+  public static List<RelCollation> calc(RelMetadataQuery mq, RelNode input,
       RexProgram program) {
-    return program.getCollations(RelMetadataQuery.collations(input));
+    return program.getCollations(mq.collations(input));
   }
 
   /** Helper method to determine a {@link Project}'s collation. */
-  public static List<RelCollation> project(RelNode input,
-      List<? extends RexNode> projects) {
-    final SortedSet<RelCollation> collations = Sets.newTreeSet();
-    final List<RelCollation> inputCollations =
-        RelMetadataQuery.collations(input);
+  public static List<RelCollation> project(RelMetadataQuery mq,
+      RelNode input, List<? extends RexNode> projects) {
+    final SortedSet<RelCollation> collations = new TreeSet<>();
+    final List<RelCollation> inputCollations = mq.collations(input);
     if (inputCollations == null || inputCollations.isEmpty()) {
       return ImmutableList.of();
     }
@@ -202,7 +216,7 @@ public class RelMdCollation {
         targetsWithMonotonicity.put(project.i, call.getOperator().getMonotonicity(binding));
       }
     }
-    final List<RelFieldCollation> fieldCollations = Lists.newArrayList();
+    final List<RelFieldCollation> fieldCollations = new ArrayList<>();
   loop:
     for (RelCollation ic : inputCollations) {
       if (ic.getFieldCollations().isEmpty()) {
@@ -251,9 +265,9 @@ public class RelMdCollation {
    * from each of its windows. Assuming (quite reasonably) that the
    * implementation does not re-order its input rows, then any collations of its
    * input are preserved. */
-  public static List<RelCollation> window(RelNode input,
+  public static List<RelCollation> window(RelMetadataQuery mq, RelNode input,
       ImmutableList<Window.Group> groups) {
-    return RelMetadataQuery.collations(input);
+    return mq.collations(input);
   }
 
   /** Helper method to determine a
@@ -274,8 +288,9 @@ public class RelMdCollation {
    *
    * <p>So, for an empty Values with 4 columns, we would emit
    * {@code (a, b, c, d), (b, c, d), (c, d), (d)}. */
-  public static List<RelCollation> values(RelDataType rowType,
-      ImmutableList<ImmutableList<RexLiteral>> tuples) {
+  public static List<RelCollation> values(RelMetadataQuery mq,
+      RelDataType rowType, ImmutableList<ImmutableList<RexLiteral>> tuples) {
+    Util.discard(mq); // for future use
     final List<RelCollation> list = Lists.newArrayList();
     final int n = rowType.getFieldCount();
     final List<Pair<RelFieldCollation, Ordering<List<RexLiteral>>>> pairs =
@@ -336,18 +351,17 @@ public class RelMdCollation {
    *
    * <p>If the inputs are sorted on other keys <em>in addition to</em> the join
    * key, the result preserves those collations too. */
-  public static List<RelCollation> mergeJoin(RelNode left, RelNode right,
+  public static List<RelCollation> mergeJoin(RelMetadataQuery mq,
+      RelNode left, RelNode right,
       ImmutableIntList leftKeys, ImmutableIntList rightKeys) {
     final ImmutableList.Builder<RelCollation> builder = ImmutableList.builder();
 
-    final ImmutableList<RelCollation> leftCollations =
-        RelMetadataQuery.collations(left);
+    final ImmutableList<RelCollation> leftCollations = mq.collations(left);
     assert RelCollations.contains(leftCollations, leftKeys)
         : "cannot merge join: left input is not sorted on left keys";
     builder.addAll(leftCollations);
 
-    final ImmutableList<RelCollation> rightCollations =
-        RelMetadataQuery.collations(right);
+    final ImmutableList<RelCollation> rightCollations = mq.collations(right);
     assert RelCollations.contains(rightCollations, rightKeys)
         : "cannot merge join: right input is not sorted on right keys";
     final int leftFieldCount = left.getRowType().getFieldCount();

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java
index 7f06443..3038260 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java
@@ -53,14 +53,11 @@ public class RelMdColumnOrigins {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Set<RelColumnOrigin> getColumnOrigins(
-      Aggregate rel,
-      int iOutputColumn) {
+  public Set<RelColumnOrigin> getColumnOrigins(Aggregate rel,
+      RelMetadataQuery mq, int iOutputColumn) {
     if (iOutputColumn < rel.getGroupCount()) {
       // Group columns pass through directly.
-      return invokeGetColumnOrigins(
-          rel.getInput(),
-          iOutputColumn);
+      return mq.getColumnOrigins(rel.getInput(), iOutputColumn);
     }
 
     if (rel.indicator) {
@@ -75,11 +72,10 @@ public class RelMdColumnOrigins {
         rel.getAggCallList().get(iOutputColumn
                 - rel.getGroupCount() - rel.getIndicatorCount());
 
-    Set<RelColumnOrigin> set = new HashSet<RelColumnOrigin>();
+    final Set<RelColumnOrigin> set = new HashSet<>();
     for (Integer iInput : call.getArgList()) {
       Set<RelColumnOrigin> inputSet =
-          invokeGetColumnOrigins(
-              rel.getInput(), iInput);
+          mq.getColumnOrigins(rel.getInput(), iInput);
       inputSet = createDerivedColumnOrigins(inputSet);
       if (inputSet != null) {
         set.addAll(inputSet);
@@ -88,25 +84,18 @@ public class RelMdColumnOrigins {
     return set;
   }
 
-  public Set<RelColumnOrigin> getColumnOrigins(
-      Join rel,
+  public Set<RelColumnOrigin> getColumnOrigins(Join rel, RelMetadataQuery mq,
       int iOutputColumn) {
     int nLeftColumns = rel.getLeft().getRowType().getFieldList().size();
     Set<RelColumnOrigin> set;
     boolean derived = false;
     if (iOutputColumn < nLeftColumns) {
-      set =
-          invokeGetColumnOrigins(
-              rel.getLeft(),
-              iOutputColumn);
+      set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn);
       if (rel.getJoinType().generatesNullsOnLeft()) {
         derived = true;
       }
     } else {
-      set =
-          invokeGetColumnOrigins(
-              rel.getRight(),
-              iOutputColumn - nLeftColumns);
+      set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns);
       if (rel.getJoinType().generatesNullsOnRight()) {
         derived = true;
       }
@@ -119,15 +108,11 @@ public class RelMdColumnOrigins {
     return set;
   }
 
-  public Set<RelColumnOrigin> getColumnOrigins(
-      SetOp rel,
-      int iOutputColumn) {
-    Set<RelColumnOrigin> set = new HashSet<RelColumnOrigin>();
+  public Set<RelColumnOrigin> getColumnOrigins(SetOp rel,
+      RelMetadataQuery mq, int iOutputColumn) {
+    final Set<RelColumnOrigin> set = new HashSet<>();
     for (RelNode input : rel.getInputs()) {
-      Set<RelColumnOrigin> inputSet =
-          invokeGetColumnOrigins(
-              input,
-              iOutputColumn);
+      Set<RelColumnOrigin> inputSet = mq.getColumnOrigins(input, iOutputColumn);
       if (inputSet == null) {
         return null;
       }
@@ -136,30 +121,25 @@ public class RelMdColumnOrigins {
     return set;
   }
 
-  public Set<RelColumnOrigin> getColumnOrigins(
-      Project rel,
-      int iOutputColumn) {
-    final RelNode child = rel.getInput();
+  public Set<RelColumnOrigin> getColumnOrigins(Project rel,
+      final RelMetadataQuery mq, int iOutputColumn) {
+    final RelNode input = rel.getInput();
     RexNode rexNode = rel.getProjects().get(iOutputColumn);
 
     if (rexNode instanceof RexInputRef) {
       // Direct reference:  no derivation added.
       RexInputRef inputRef = (RexInputRef) rexNode;
-      return invokeGetColumnOrigins(
-          child,
-          inputRef.getIndex());
+      return mq.getColumnOrigins(input, inputRef.getIndex());
     }
 
     // Anything else is a derivation, possibly from multiple
     // columns.
-    final Set<RelColumnOrigin> set = new HashSet<RelColumnOrigin>();
+    final Set<RelColumnOrigin> set = new HashSet<>();
     RexVisitor visitor =
         new RexVisitorImpl<Void>(true) {
           public Void visitInputRef(RexInputRef inputRef) {
             Set<RelColumnOrigin> inputSet =
-                invokeGetColumnOrigins(
-                    child,
-                    inputRef.getIndex());
+                mq.getColumnOrigins(input, inputRef.getIndex());
             if (inputSet != null) {
               set.addAll(inputSet);
             }
@@ -171,34 +151,24 @@ public class RelMdColumnOrigins {
     return createDerivedColumnOrigins(set);
   }
 
-  public Set<RelColumnOrigin> getColumnOrigins(
-      Filter rel,
-      int iOutputColumn) {
-    return invokeGetColumnOrigins(
-        rel.getInput(),
-        iOutputColumn);
+  public Set<RelColumnOrigin> getColumnOrigins(Filter rel,
+      RelMetadataQuery mq, int iOutputColumn) {
+    return mq.getColumnOrigins(rel.getInput(), iOutputColumn);
   }
 
-  public Set<RelColumnOrigin> getColumnOrigins(
-      Sort rel,
+  public Set<RelColumnOrigin> getColumnOrigins(Sort rel, RelMetadataQuery mq,
       int iOutputColumn) {
-    return invokeGetColumnOrigins(
-        rel.getInput(),
-        iOutputColumn);
+    return mq.getColumnOrigins(rel.getInput(), iOutputColumn);
   }
 
-  public Set<RelColumnOrigin> getColumnOrigins(
-      Exchange rel,
-      int iOutputColumn) {
-    return invokeGetColumnOrigins(
-        rel.getInput(),
-        iOutputColumn);
+  public Set<RelColumnOrigin> getColumnOrigins(Exchange rel,
+      RelMetadataQuery mq, int iOutputColumn) {
+    return mq.getColumnOrigins(rel.getInput(), iOutputColumn);
   }
 
-  public Set<RelColumnOrigin> getColumnOrigins(
-      TableFunctionScan rel,
-      int iOutputColumn) {
-    Set<RelColumnOrigin> set = new HashSet<RelColumnOrigin>();
+  public Set<RelColumnOrigin> getColumnOrigins(TableFunctionScan rel,
+      RelMetadataQuery mq, int iOutputColumn) {
+    final Set<RelColumnOrigin> set = new HashSet<>();
     Set<RelColumnMapping> mappings = rel.getColumnMappings();
     if (mappings == null) {
       if (rel.getInputs().size() > 0) {
@@ -216,10 +186,9 @@ public class RelMdColumnOrigins {
       if (mapping.iOutputColumn != iOutputColumn) {
         continue;
       }
-      Set<RelColumnOrigin> origins =
-          invokeGetColumnOrigins(
-              rel.getInputs().get(mapping.iInputRel),
-              mapping.iInputColumn);
+      final RelNode input = rel.getInputs().get(mapping.iInputRel);
+      final int column = mapping.iInputColumn;
+      Set<RelColumnOrigin> origins = mq.getColumnOrigins(input, column);
       if (origins == null) {
         return null;
       }
@@ -232,9 +201,8 @@ public class RelMdColumnOrigins {
   }
 
   // Catch-all rule when none of the others apply.
-  public Set<RelColumnOrigin> getColumnOrigins(
-      RelNode rel,
-      int iOutputColumn) {
+  public Set<RelColumnOrigin> getColumnOrigins(RelNode rel,
+      RelMetadataQuery mq, int iOutputColumn) {
     // NOTE jvs 28-Mar-2006: We may get this wrong for a physical table
     // expression which supports projections.  In that case,
     // it's up to the plugin writer to override with the
@@ -245,7 +213,7 @@ public class RelMdColumnOrigins {
       return null;
     }
 
-    Set<RelColumnOrigin> set = new HashSet<RelColumnOrigin>();
+    final Set<RelColumnOrigin> set = new HashSet<>();
 
     RelOptTable table = rel.getTable();
     if (table == null) {
@@ -267,18 +235,12 @@ public class RelMdColumnOrigins {
     return set;
   }
 
-  protected Set<RelColumnOrigin> invokeGetColumnOrigins(
-      RelNode rel,
-      int iOutputColumn) {
-    return RelMetadataQuery.getColumnOrigins(rel, iOutputColumn);
-  }
-
   private Set<RelColumnOrigin> createDerivedColumnOrigins(
       Set<RelColumnOrigin> inputSet) {
     if (inputSet == null) {
       return null;
     }
-    Set<RelColumnOrigin> set = new HashSet<RelColumnOrigin>();
+    final Set<RelColumnOrigin> set = new HashSet<>();
     for (RelColumnOrigin rco : inputSet) {
       RelColumnOrigin derived =
           new RelColumnOrigin(


[17/50] [abbrv] calcite git commit: Simplify RexProgram, in particular "(NOT CASE ... END) IS TRUE", which occurs in NOT IN

Posted by jh...@apache.org.
Simplify RexProgram, in particular "(NOT CASE ... END) IS TRUE", which occurs in NOT IN


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

Branch: refs/heads/branch-release
Commit: 7837e64c3294aa776ca38cba3f756550af3494ab
Parents: 4762b88
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Aug 19 16:25:30 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 .../adapter/enumerable/EnumerableCalc.java      |   3 +
 .../calcite/adapter/enumerable/RexImpTable.java |   2 +-
 .../calcite/rel/rules/ProjectToWindowRule.java  |   4 +-
 .../rel/rules/ReduceExpressionsRule.java        |   2 +-
 .../java/org/apache/calcite/rex/RexCall.java    |  10 ++
 .../java/org/apache/calcite/rex/RexProgram.java |  41 ++++---
 .../apache/calcite/rex/RexProgramBuilder.java   | 111 +++++++++--------
 .../java/org/apache/calcite/rex/RexUtil.java    | 118 ++++++++++++++++---
 .../java/org/apache/calcite/sql/SqlKind.java    |  26 ++++
 .../java/org/apache/calcite/sql/SqlMerge.java   |   9 +-
 .../main/java/org/apache/calcite/util/Pair.java |   8 +-
 .../org/apache/calcite/test/RexProgramTest.java | 117 ++++++++++++++++--
 12 files changed, 340 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
index 603c7b1..ce1f642 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
@@ -144,6 +144,9 @@ public class EnumerableCalc extends Calc implements EnumerableRel {
                 BuiltInMethod.ENUMERATOR_CURRENT.method),
             inputJavaType);
 
+    final RexProgram program =
+        this.program.normalize(getCluster().getRexBuilder(), true);
+
     BlockStatement moveNextBody;
     if (program.getCondition() == null) {
       moveNextBody =

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/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 20872b6..511584b 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
@@ -1887,7 +1887,7 @@ public class RexImpTable {
             negate == seek,
             translator.translate(
                 operands.get(0),
-                seek ? NullAs.FALSE : NullAs.TRUE));
+                negate == seek ? NullAs.TRUE : NullAs.FALSE));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java
index fe6334e..ed4f610 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java
@@ -35,7 +35,6 @@ import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexProgram;
-import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.rex.RexWindow;
 import org.apache.calcite.tools.RelBuilder;
@@ -199,8 +198,7 @@ public abstract class ProjectToWindowRule extends RelOptRule {
         protected RelNode makeRel(RelOptCluster cluster, RelTraitSet traitSet,
             RelBuilder relBuilder, RelNode input, RexProgram program) {
           assert !program.containsAggs();
-          program = RexProgramBuilder.normalize(cluster.getRexBuilder(),
-              program);
+          program = program.normalize(cluster.getRexBuilder(), false);
           return super.makeRel(cluster, traitSet, relBuilder, input, program);
         }
       },

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 17d5cb0..759748c 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -663,7 +663,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
    * <p>We have a loose definition of 'predicate': any boolean expression will
    * do, except CASE. For example '(CASE ...) = 5' or '(CASE ...) IS NULL'.
    */
-  protected static RexCall pushPredicateIntoCase(RexCall call) {
+  public static RexCall pushPredicateIntoCase(RexCall call) {
     if (call.getType().getSqlTypeName() != SqlTypeName.BOOLEAN) {
       return call;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/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 8271005..7cf8255 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -118,6 +118,11 @@ public class RexCall extends RexNode {
     switch (getKind()) {
     case IS_NOT_NULL:
       return !operands.get(0).getType().isNullable();
+    case IS_NOT_FALSE:
+    case NOT:
+      return operands.get(0).isAlwaysFalse();
+    case IS_NOT_TRUE:
+    case IS_FALSE:
     case CAST:
       return operands.get(0).isAlwaysTrue();
     default:
@@ -129,6 +134,11 @@ public class RexCall extends RexNode {
     switch (getKind()) {
     case IS_NULL:
       return !operands.get(0).getType().isNullable();
+    case IS_NOT_TRUE:
+    case NOT:
+      return operands.get(0).isAlwaysTrue();
+    case IS_NOT_FALSE:
+    case IS_TRUE:
     case CAST:
       return operands.get(0).isAlwaysFalse();
     default:

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/core/src/main/java/org/apache/calcite/rex/RexProgram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgram.java b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
index 58cdc40..664c092 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgram.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
@@ -42,6 +42,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 /**
  * A collection of expressions which read inputs, compute output expressions,
@@ -351,8 +352,8 @@ public class RexProgram {
           "field type mismatch: " + rowType + " vs. " + outputRowType);
     }
     final List<RelDataTypeField> fields = rowType.getFieldList();
-    final List<RexLocalRef> projectRefs = new ArrayList<RexLocalRef>();
-    final List<RexInputRef> refs = new ArrayList<RexInputRef>();
+    final List<RexLocalRef> projectRefs = new ArrayList<>();
+    final List<RexInputRef> refs = new ArrayList<>();
     for (int i = 0; i < fields.size(); i++) {
       final RexInputRef ref = RexInputRef.of(i, fields);
       refs.add(ref);
@@ -462,14 +463,14 @@ public class RexProgram {
         return litmus.fail(null);
       }
     }
-    for (int i = 0; i < projects.size(); i++) {
-      projects.get(i).accept(checker);
+    for (RexLocalRef project : projects) {
+      project.accept(checker);
       if (checker.failCount > 0) {
         return litmus.fail(null);
       }
     }
-    for (int i = 0; i < exprs.size(); i++) {
-      exprs.get(i).accept(checker);
+    for (RexNode expr : exprs) {
+      expr.accept(checker);
       if (checker.failCount > 0) {
         return litmus.fail(null);
       }
@@ -511,10 +512,7 @@ public class RexProgram {
    * @return expanded form
    */
   public RexNode expandLocalRef(RexLocalRef ref) {
-    // TODO jvs 19-Apr-2006:  assert that ref is part of
-    // this program
-    ExpansionShuttle shuttle = new ExpansionShuttle();
-    return ref.accept(shuttle);
+    return ref.accept(new ExpansionShuttle(exprs));
   }
 
   /** Splits this program into a list of project expressions and a list of
@@ -540,7 +538,7 @@ public class RexProgram {
    * mutable.
    */
   public List<RelCollation> getCollations(List<RelCollation> inputCollations) {
-    List<RelCollation> outputCollations = new ArrayList<RelCollation>(1);
+    List<RelCollation> outputCollations = new ArrayList<>(1);
     deduceCollations(
         outputCollations,
         inputRowType.getFieldCount(), projects,
@@ -568,8 +566,7 @@ public class RexProgram {
     }
   loop:
     for (RelCollation collation : inputCollations) {
-      final ArrayList<RelFieldCollation> fieldCollations =
-          new ArrayList<RelFieldCollation>(0);
+      final List<RelFieldCollation> fieldCollations = new ArrayList<>(0);
       for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
         final int source = fieldCollation.getFieldIndex();
         final int target = targets[source];
@@ -741,8 +738,8 @@ public class RexProgram {
    *
    * @return set of correlation variable names
    */
-  public HashSet<String> getCorrelVariableNames() {
-    final HashSet<String> paramIdSet = new HashSet<String>();
+  public Set<String> getCorrelVariableNames() {
+    final Set<String> paramIdSet = new HashSet<>();
     RexUtil.apply(
         new RexVisitorImpl<Void>(true) {
           public Void visitCorrelVariable(
@@ -790,7 +787,7 @@ public class RexProgram {
     assert isValid(Litmus.THROW);
     final RexProgramBuilder builder =
         RexProgramBuilder.create(rexBuilder, inputRowType, exprs, projects,
-            condition, outputRowType, simplify);
+            condition, outputRowType, true, simplify);
     return builder.getProgram(false);
   }
 
@@ -840,9 +837,15 @@ public class RexProgram {
    * A RexShuttle used in the implementation of
    * {@link RexProgram#expandLocalRef}.
    */
-  private class ExpansionShuttle extends RexShuttle {
+  static class ExpansionShuttle extends RexShuttle {
+    private final List<RexNode> exprs;
+
+    public ExpansionShuttle(List<RexNode> exprs) {
+      this.exprs = exprs;
+    }
+
     public RexNode visitLocalRef(RexLocalRef localRef) {
-      RexNode tree = getExprList().get(localRef.getIndex());
+      RexNode tree = exprs.get(localRef.getIndex());
       return tree.accept(this);
     }
   }
@@ -886,7 +889,7 @@ public class RexProgram {
     }
 
     public RexNode visitCall(RexCall call) {
-      final List<RexNode> newOperands = new ArrayList<RexNode>();
+      final List<RexNode> newOperands = new ArrayList<>();
       for (RexNode operand : call.getOperands()) {
         newOperands.add(operand.accept(this));
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
index c292e56..5f7d3ef 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
@@ -41,13 +41,12 @@ public class RexProgramBuilder {
 
   private final RexBuilder rexBuilder;
   private final RelDataType inputRowType;
-  private final List<RexNode> exprList = new ArrayList<RexNode>();
+  private final List<RexNode> exprList = new ArrayList<>();
   private final Map<Pair<String, String>, RexLocalRef> exprMap =
-      new HashMap<Pair<String, String>, RexLocalRef>();
-  private final List<RexLocalRef> localRefList = new ArrayList<RexLocalRef>();
-  private final List<RexLocalRef> projectRefList =
-      new ArrayList<RexLocalRef>();
-  private final List<String> projectNameList = new ArrayList<String>();
+      new HashMap<>();
+  private final List<RexLocalRef> localRefList = new ArrayList<>();
+  private final List<RexLocalRef> projectRefList = new ArrayList<>();
+  private final List<String> projectNameList = new ArrayList<>();
   private RexLocalRef conditionRef = null;
   private boolean validating;
 
@@ -78,19 +77,21 @@ public class RexProgramBuilder {
    * @param rexBuilder     Rex builder
    * @param inputRowType   Input row type
    * @param exprList       Common expressions
-   * @param projectRefList Projections
-   * @param conditionRef   Condition, or null
+   * @param projectList    Projections
+   * @param condition      Condition, or null
    * @param outputRowType  Output row type
    * @param normalize      Whether to normalize
+   * @param simplify       Whether to simplify
    */
   private RexProgramBuilder(
       RexBuilder rexBuilder,
       final RelDataType inputRowType,
       final List<RexNode> exprList,
-      final List<RexLocalRef> projectRefList,
-      final RexLocalRef conditionRef,
+      final Iterable<? extends RexNode> projectList,
+      RexNode condition,
       final RelDataType outputRowType,
-      boolean normalize) {
+      boolean normalize,
+      boolean simplify) {
     this(inputRowType, rexBuilder);
 
     // Create a shuttle for registering input expressions.
@@ -106,24 +107,38 @@ public class RexProgramBuilder {
       }
     }
 
+    final RexShuttle expander = new RexProgram.ExpansionShuttle(exprList);
+
     // Register project expressions
     // and create a named project item.
     final List<RelDataTypeField> fieldList = outputRowType.getFieldList();
-    for (Pair<RexLocalRef, RelDataTypeField> pair
-        : Pair.zip(projectRefList, fieldList)) {
-      final RexLocalRef projectRef = pair.left;
+    for (Pair<? extends RexNode, RelDataTypeField> pair
+        : Pair.zip(projectList, fieldList)) {
+      final RexNode project;
+      if (simplify) {
+        project = RexUtil.simplify(rexBuilder, pair.left.accept(expander));
+      } else {
+        project = pair.left;
+      }
       final String name = pair.right.getName();
-      final int oldIndex = projectRef.getIndex();
-      final RexNode expr = exprList.get(oldIndex);
-      final RexLocalRef ref = (RexLocalRef) expr.accept(shuttle);
+      final RexLocalRef ref = (RexLocalRef) project.accept(shuttle);
       addProject(ref.getIndex(), name);
     }
 
     // Register the condition, if there is one.
-    if (conditionRef != null) {
-      final RexNode expr = exprList.get(conditionRef.getIndex());
-      final RexLocalRef ref = (RexLocalRef) expr.accept(shuttle);
-      addCondition(ref);
+    if (condition != null) {
+      if (simplify) {
+        condition = RexUtil.simplify(rexBuilder,
+            rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE,
+                condition.accept(expander)));
+        if (condition.isAlwaysTrue()) {
+          condition = null;
+        }
+      }
+      if (condition != null) {
+        final RexLocalRef ref = (RexLocalRef) condition.accept(shuttle);
+        addCondition(ref);
+      }
     }
   }
 
@@ -463,7 +478,8 @@ public class RexProgramBuilder {
         projectRefs,
         conditionRef,
         outputRowType,
-        normalize);
+        normalize,
+        false);
   }
 
   /**
@@ -494,28 +510,37 @@ public class RexProgramBuilder {
    * @param rexBuilder     Rex builder
    * @param inputRowType   Input row type
    * @param exprList       Common expressions
-   * @param projectRefList Projections
-   * @param conditionRef   Condition, or null
+   * @param projectList    Projections
+   * @param condition      Condition, or null
    * @param outputRowType  Output row type
    * @param normalize      Whether to normalize
+   * @param simplify       Whether to simplify expressions
    * @return A program builder
    */
   public static RexProgramBuilder create(
       RexBuilder rexBuilder,
       final RelDataType inputRowType,
       final List<RexNode> exprList,
-      final List<RexLocalRef> projectRefList,
-      final RexLocalRef conditionRef,
+      final List<? extends RexNode> projectList,
+      final RexNode condition,
+      final RelDataType outputRowType,
+      boolean normalize,
+      boolean simplify) {
+    return new RexProgramBuilder(rexBuilder, inputRowType, exprList,
+        projectList, condition, outputRowType, normalize, simplify);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public static RexProgramBuilder create(
+      RexBuilder rexBuilder,
+      final RelDataType inputRowType,
+      final List<RexNode> exprList,
+      final List<? extends RexNode> projectList,
+      final RexNode condition,
       final RelDataType outputRowType,
       boolean normalize) {
-    return new RexProgramBuilder(
-        rexBuilder,
-        inputRowType,
-        exprList,
-        projectRefList,
-        conditionRef,
-        outputRowType,
-        normalize);
+    return create(rexBuilder, inputRowType, exprList, projectList, condition,
+        outputRowType, normalize, false);
   }
 
   /**
@@ -557,21 +582,11 @@ public class RexProgramBuilder {
     return progBuilder;
   }
 
-  /**
-   * Normalizes a program.
-   *
-   * @param rexBuilder Rex builder
-   * @param program    Program
-   * @return Normalized program
-   */
+  @Deprecated // to be removed before 2.0
   public static RexProgram normalize(
       RexBuilder rexBuilder,
       RexProgram program) {
-    // Normalize program by creating program builder from the program, then
-    // converting to a program. getProgram does not need to normalize
-    // because the builder was normalized on creation.
-    return forProgram(program, rexBuilder, true)
-        .getProgram(false);
+    return program.normalize(rexBuilder, false);
   }
 
   /**
@@ -601,7 +616,7 @@ public class RexProgramBuilder {
     // register the result.
     // REVIEW jpham 28-Apr-2006: if the user shuttle rewrites an input
     // expression, then input references may change
-    List<RexLocalRef> newRefs = new ArrayList<RexLocalRef>(exprList.size());
+    List<RexLocalRef> newRefs = new ArrayList<>(exprList.size());
     RexShuttle refShuttle = new UpdateRefShuttle(newRefs);
     int i = 0;
     for (RexNode expr : exprList) {
@@ -753,7 +768,7 @@ public class RexProgramBuilder {
 
   private List<RexLocalRef> registerProjectsAndCondition(RexProgram program) {
     final List<RexNode> exprList = program.getExprList();
-    final List<RexLocalRef> projectRefList = new ArrayList<RexLocalRef>();
+    final List<RexLocalRef> projectRefList = new ArrayList<>();
     final RexShuttle shuttle = new RegisterOutputShuttle(exprList);
 
     // For each project, lookup the expr and expand it so it is in terms of

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 8e8a79c..b2de640 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -1087,9 +1087,10 @@ public class RexUtil {
 
   /** Converts an expression to disjunctive normal form (DNF).
    *
-   * <p>DNF: It is a form of logical formula which is disjunction of conjunctive clauses</p>
+   * <p>DNF: It is a form of logical formula which is disjunction of conjunctive
+   * clauses.
    *
-   * <p>All logicl formulas can be converted into DNF.</p>
+   * <p>All logical formulas can be converted into DNF.
    *
    * <p>The following expression is in DNF:
    *
@@ -1280,19 +1281,100 @@ public class RexUtil {
       return simplifyAnd(rexBuilder, (RexCall) e);
     case OR:
       return simplifyOr(rexBuilder, (RexCall) e);
+    case NOT:
+      return simplifyNot(rexBuilder, (RexCall) e);
     case CASE:
       return simplifyCase(rexBuilder, (RexCall) e);
+    }
+    switch (e.getKind()) {
     case IS_NULL:
-      return ((RexCall) e).getOperands().get(0).getType().isNullable()
-          ? e : rexBuilder.makeLiteral(false);
     case IS_NOT_NULL:
-      return ((RexCall) e).getOperands().get(0).getType().isNullable()
-          ? e : rexBuilder.makeLiteral(true);
+    case IS_TRUE:
+    case IS_NOT_TRUE:
+    case IS_FALSE:
+    case IS_NOT_FALSE:
+      assert e instanceof RexCall;
+      return simplifyIs(rexBuilder, (RexCall) e);
     default:
       return e;
     }
   }
 
+  private static RexNode simplifyNot(RexBuilder rexBuilder, RexCall call) {
+    final RexNode a = call.getOperands().get(0);
+    switch (a.getKind()) {
+    case NOT:
+      // NOT NOT x ==> x
+      return simplify(rexBuilder, ((RexCall) a).getOperands().get(0));
+    }
+    final SqlKind negateKind = a.getKind().negate();
+    if (a.getKind() != negateKind) {
+      return simplify(rexBuilder,
+          rexBuilder.makeCall(op(negateKind),
+              ImmutableList.of(((RexCall) a).getOperands().get(0))));
+    }
+    return call;
+  }
+
+  private static RexNode simplifyIs(RexBuilder rexBuilder, RexCall call) {
+    final SqlKind kind = call.getKind();
+    final RexNode a = call.getOperands().get(0);
+    if (!a.getType().isNullable()) {
+      switch (kind) {
+      case IS_NULL:
+      case IS_NOT_NULL:
+        // x IS NULL ==> FALSE (if x is not nullable)
+        // x IS NOT NULL ==> TRUE (if x is not nullable)
+        return rexBuilder.makeLiteral(kind == SqlKind.IS_NOT_NULL);
+      case IS_TRUE:
+      case IS_NOT_FALSE:
+        // x IS TRUE ==> x (if x is not nullable)
+        // x IS NOT FALSE ==> x (if x is not nullable)
+        return simplify(rexBuilder, a);
+      case IS_FALSE:
+      case IS_NOT_TRUE:
+        // x IS NOT TRUE ==> NOT x (if x is not nullable)
+        // x IS FALSE ==> NOT x (if x is not nullable)
+        return simplify(rexBuilder,
+            rexBuilder.makeCall(SqlStdOperatorTable.NOT, a));
+      }
+    }
+    switch (a.getKind()) {
+    case NOT:
+      // NOT x IS TRUE ==> x IS NOT TRUE
+      // Similarly for IS NOT TRUE, IS FALSE, etc.
+      return simplify(rexBuilder,
+          rexBuilder.makeCall(op(kind.negate()),
+              ((RexCall) a).getOperands().get(0)));
+    }
+    RexNode a2 = simplify(rexBuilder, a);
+    if (a != a2) {
+      return rexBuilder.makeCall(op(kind), ImmutableList.of(a2));
+    }
+    return call;
+  }
+
+  private static SqlOperator op(SqlKind kind) {
+    switch (kind) {
+    case IS_FALSE:
+      return SqlStdOperatorTable.IS_FALSE;
+    case IS_TRUE:
+      return SqlStdOperatorTable.IS_TRUE;
+    case IS_UNKNOWN:
+      return SqlStdOperatorTable.IS_UNKNOWN;
+    case IS_NULL:
+      return SqlStdOperatorTable.IS_NULL;
+    case IS_NOT_FALSE:
+      return SqlStdOperatorTable.IS_NOT_FALSE;
+    case IS_NOT_TRUE:
+      return SqlStdOperatorTable.IS_NOT_TRUE;
+    case IS_NOT_NULL:
+      return SqlStdOperatorTable.IS_NOT_NULL;
+    default:
+      throw new AssertionError(kind);
+    }
+  }
+
   private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call) {
     final List<RexNode> operands = call.getOperands();
     final List<RexNode> newOperands = new ArrayList<>();
@@ -1379,11 +1461,13 @@ public class RexUtil {
         --i;
         break;
       case LITERAL:
-        if (!RexLiteral.booleanValue(term)) {
-          return term; // false
-        } else {
-          terms.remove(i);
-          --i;
+        if (!RexLiteral.isNullLiteral(term)) {
+          if (!RexLiteral.booleanValue(term)) {
+            return term; // false
+          } else {
+            terms.remove(i);
+            --i;
+          }
         }
       }
     }
@@ -1420,11 +1504,13 @@ public class RexUtil {
       final RexNode term = terms.get(i);
       switch (term.getKind()) {
       case LITERAL:
-        if (RexLiteral.booleanValue(term)) {
-          return term; // true
-        } else {
-          terms.remove(i);
-          --i;
+        if (!RexLiteral.isNullLiteral(term)) {
+          if (RexLiteral.booleanValue(term)) {
+            return term; // true
+          } else {
+            terms.remove(i);
+            --i;
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/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 78a9d7d..37a757d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -907,6 +907,32 @@ 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}. */
+  public SqlKind negate() {
+    switch (this) {
+    case IS_TRUE:
+      return IS_NOT_TRUE;
+    case IS_FALSE:
+      return IS_NOT_FALSE;
+    case IS_NULL:
+      return IS_NOT_NULL;
+    case IS_NOT_TRUE:
+      return IS_TRUE;
+    case IS_NOT_FALSE:
+      return IS_FALSE;
+    case IS_NOT_NULL:
+      return IS_NULL;
+    case IS_DISTINCT_FROM:
+      return IS_NOT_DISTINCT_FROM;
+    case IS_NOT_DISTINCT_FROM:
+      return IS_DISTINCT_FROM;
+    default:
+      return this;
+    }
+  }
+
   /**
    * Returns whether this {@code SqlKind} belongs to a given category.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlMerge.java b/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
index 77be4e7..af38c39 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
@@ -23,7 +23,6 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.Pair;
 
-import java.util.Iterator;
 import java.util.List;
 
 /**
@@ -80,7 +79,8 @@ public class SqlMerge extends SqlCall {
   @Override public void setOperand(int i, SqlNode operand) {
     switch (i) {
     case 0:
-      targetTable = (SqlIdentifier) operand;
+      assert operand instanceof SqlIdentifier;
+      targetTable = operand;
       break;
     case 1:
       condition = operand;
@@ -194,11 +194,6 @@ public class SqlMerge extends SqlCall {
               "SET",
               "");
 
-      Iterator targetColumnIter =
-          updateCall.getTargetColumnList().getList().iterator();
-      Iterator sourceExpressionIter =
-          updateCall.getSourceExpressionList().getList().iterator();
-
       for (Pair<SqlNode, SqlNode> pair : Pair.zip(
           updateCall.targetColumnList, updateCall.sourceExpressionList)) {
         writer.sep(",");

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/core/src/main/java/org/apache/calcite/util/Pair.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Pair.java b/core/src/main/java/org/apache/calcite/util/Pair.java
index a01e9e7..f4f5b8d 100644
--- a/core/src/main/java/org/apache/calcite/util/Pair.java
+++ b/core/src/main/java/org/apache/calcite/util/Pair.java
@@ -216,12 +216,12 @@ public class Pair<T1, T2>
    * @return Iterable over pairs
    */
   public static <K, V> Iterable<Pair<K, V>> zip(
-      final Iterable<K> ks,
-      final Iterable<V> vs) {
+      final Iterable<? extends K> ks,
+      final Iterable<? extends V> vs) {
     return new Iterable<Pair<K, V>>() {
       public Iterator<Pair<K, V>> iterator() {
-        final Iterator<K> kIterator = ks.iterator();
-        final Iterator<V> vIterator = vs.iterator();
+        final Iterator<? extends K> kIterator = ks.iterator();
+        final Iterator<? extends V> vIterator = vs.iterator();
 
         return new Iterator<Pair<K, V>>() {
           public boolean hasNext() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/7837e64c/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 20162dd..7b50427 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -153,10 +153,7 @@ public class RexProgramTest {
     // Normalize the program using the RexProgramBuilder.normalize API.
     // Note that unused expression '77' is eliminated, input refs (e.g. $0)
     // become local refs (e.g. $t0), and constants are assigned to locals.
-    final RexProgram normalizedProgram =
-        RexProgramBuilder.normalize(
-            rexBuilder,
-            program);
+    final RexProgram normalizedProgram = program.normalize(rexBuilder, false);
     final String normalizedProgramString = normalizedProgram.toString();
     TestUtil.assertEqualsVerbose(
         "(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[1], "
@@ -201,6 +198,47 @@ public class RexProgramTest {
   }
 
   /**
+   * Tests how the condition is simplified.
+   */
+  @Test public void testSimplifyCondition() {
+    final RexProgram program = createProg(3).getProgram(false);
+    assertThat(program.toString(),
+        is("(expr#0..1=[{inputs}], expr#2=[+($0, 1)], expr#3=[77], "
+            + "expr#4=[+($0, $1)], expr#5=[+($0, 1)], expr#6=[+($0, $t5)], "
+            + "expr#7=[+($t4, $t2)], expr#8=[5], expr#9=[>($t2, $t8)], "
+            + "expr#10=[true], expr#11=[IS NOT NULL($t5)], expr#12=[false], "
+            + "expr#13=[null], expr#14=[CASE($t9, $t10, $t11, $t12, $t13)], "
+            + "expr#15=[NOT($t14)], a=[$t7], b=[$t6], $condition=[$t15])"));
+
+    assertThat(program.normalize(rexBuilder, true).toString(),
+        is("(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[1], "
+            + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
+            + "expr#6=[+($t0, $t4)], expr#7=[5], expr#8=[>($t4, $t7)], "
+            + "expr#9=[NOT($t8)], a=[$t5], b=[$t6], $condition=[$t9])"));
+  }
+
+  /**
+   * Tests how the condition is simplified.
+   */
+  @Test public void testSimplifyCondition2() {
+    final RexProgram program = createProg(4).getProgram(false);
+    assertThat(program.toString(),
+        is("(expr#0..1=[{inputs}], expr#2=[+($0, 1)], expr#3=[77], "
+            + "expr#4=[+($0, $1)], expr#5=[+($0, 1)], expr#6=[+($0, $t5)], "
+            + "expr#7=[+($t4, $t2)], expr#8=[5], expr#9=[>($t2, $t8)], "
+            + "expr#10=[true], expr#11=[IS NOT NULL($t5)], expr#12=[false], "
+            + "expr#13=[null], expr#14=[CASE($t9, $t10, $t11, $t12, $t13)], "
+            + "expr#15=[NOT($t14)], expr#16=[IS TRUE($t15)], a=[$t7], b=[$t6], "
+            + "$condition=[$t16])"));
+
+    assertThat(program.normalize(rexBuilder, true).toString(),
+        is("(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], expr#3=[1], "
+            + "expr#4=[+($t0, $t3)], expr#5=[+($t2, $t4)], "
+            + "expr#6=[+($t0, $t4)], expr#7=[5], expr#8=[>($t4, $t7)], "
+            + "expr#9=[NOT($t8)], a=[$t5], b=[$t6], $condition=[$t9])"));
+  }
+
+  /**
    * Checks translation of AND(x, x).
    */
   @Test public void testDuplicateAnd() {
@@ -226,10 +264,16 @@ public class RexProgramTest {
    * from t(x, y)</code>
    * <li><code>select (x + y) + (x + 1) as a, (x + x) as b from t(x, y)
    * where ((x + y) > 1) and ((x + y) > 1)</code>
+   * <li><code>select (x + y) + (x + 1) as a, (x + x) as b from t(x, y)
+   * where not case
+   *           when x + 1 > 5 then true
+   *           when y is null then null
+   *           else false
+   *           end</code>
    * </ol>
    */
   private RexProgramBuilder createProg(int variant) {
-    assert variant == 0 || variant == 1 || variant == 2;
+    assert variant >= 0 && variant <= 4;
     List<RelDataType> types =
         Arrays.asList(
             typeFactory.createSqlType(SqlTypeName.INTEGER),
@@ -243,8 +287,8 @@ public class RexProgramTest {
     // $t2 = $t0 + 1 (i.e. x + 1)
     final RexNode i0 = rexBuilder.makeInputRef(
         types.get(0), 0);
-    final RexLiteral c1 = rexBuilder.makeExactLiteral(
-        BigDecimal.ONE);
+    final RexLiteral c1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
+    final RexLiteral c5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(5L));
     RexLocalRef t2 =
         builder.addExpr(
             rexBuilder.makeCall(
@@ -269,6 +313,7 @@ public class RexProgramTest {
                 i0,
                 i1));
     RexLocalRef t5;
+    final RexLocalRef t1;
     switch (variant) {
     case 0:
     case 2:
@@ -278,10 +323,13 @@ public class RexProgramTest {
               SqlStdOperatorTable.PLUS,
               i0,
               i0));
+      t1 = null;
       break;
     case 1:
+    case 3:
+    case 4:
       // $tx = $t0 + 1
-      RexLocalRef tx =
+      t1 =
           builder.addExpr(
               rexBuilder.makeCall(
                   SqlStdOperatorTable.PLUS,
@@ -293,7 +341,7 @@ public class RexProgramTest {
               rexBuilder.makeCall(
                   SqlStdOperatorTable.PLUS,
                   i0,
-                  tx));
+                  t1));
       break;
     default:
       throw Util.newInternal("unexpected variant " + variant);
@@ -308,16 +356,19 @@ public class RexProgramTest {
     builder.addProject(t6.getIndex(), "a");
     builder.addProject(t5.getIndex(), "b");
 
-    if (variant == 2) {
+    final RexLocalRef t7;
+    final RexLocalRef t8;
+    switch (variant) {
+    case 2:
       // $t7 = $t4 > $i0 (i.e. (x + y) > 0)
-      RexLocalRef t7 =
+      t7 =
           builder.addExpr(
               rexBuilder.makeCall(
                   SqlStdOperatorTable.GREATER_THAN,
                   t4,
                   i0));
       // $t8 = $t7 AND $t7
-      RexLocalRef t8 =
+      t8 =
           builder.addExpr(
               rexBuilder.makeCall(
                   SqlStdOperatorTable.AND,
@@ -325,6 +376,48 @@ public class RexProgramTest {
                   t7));
       builder.addCondition(t8);
       builder.addCondition(t7);
+      break;
+    case 3:
+    case 4:
+      // $t7 = 5
+      t7 = builder.addExpr(c5);
+      // $t8 = $t2 > $t7 (i.e. (x + 1) > 5)
+      t8 =
+          builder.addExpr(
+              rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t2, t7));
+      // $t9 = true
+      final RexLocalRef t9 =
+          builder.addExpr(rexBuilder.makeLiteral(true));
+      // $t10 = $t1 is not null (i.e. y is not null)
+      assert t1 != null;
+      final RexLocalRef t10 =
+          builder.addExpr(
+              rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, t1));
+      // $t11 = false
+      final RexLocalRef t11 =
+          builder.addExpr(rexBuilder.makeLiteral(false));
+      // $t12 = unknown
+      final RexLocalRef t12 =
+          builder.addExpr(rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN));
+      // $t13 = case when $t8 then $t9 when $t10 then $t11 else $t12 end
+      final RexLocalRef t13 =
+          builder.addExpr(
+              rexBuilder.makeCall(SqlStdOperatorTable.CASE,
+                  t8, t9, t10, t11, t12));
+      // $t14 = not $t13 (i.e. not case ... end)
+      final RexLocalRef t14 =
+          builder.addExpr(
+              rexBuilder.makeCall(SqlStdOperatorTable.NOT, t13));
+      // don't add 't14 is true' - that is implicit
+      if (variant == 3) {
+        builder.addCondition(t14);
+      } else {
+        // $t15 = $14 is true
+        final RexLocalRef t15 =
+            builder.addExpr(
+                rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, t14));
+        builder.addCondition(t15);
+      }
     }
     return builder;
   }


[45/50] [abbrv] calcite git commit: [CALCITE-978] Enable customizing constant folding rule behavior when a Filter simplifies to false (Jason Altekruse)

Posted by jh...@apache.org.
[CALCITE-978] Enable customizing constant folding rule behavior when a Filter simplifies to false (Jason Altekruse)

For static schema systems, a Filter that is always false or null can be
replaced by a Values operator that produces no rows, as the schema
information can just be taken from the input Rel. In dynamic schema
environments, the filter might have an unknown input type, in these cases
they must define a system specific alternative to a Values operator, such
as inserting a Limit 0 instead of a Filter on top of the original input.

This change enables this behavior to be customized by a subclass of the
constant reduction rules.

Close apache/calcite#183


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

Branch: refs/heads/branch-release
Commit: 898fdfc2720770ec4752fb702ce0aae1521aa07d
Parents: ee283ca
Author: Jason Altekruse <al...@gmail.com>
Authored: Fri Apr 10 15:39:54 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Jan 14 13:52:01 2016 -0800

----------------------------------------------------------------------
 .../rel/rules/ReduceExpressionsRule.java        | 51 ++++++++++++++++++--
 1 file changed, 48 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/898fdfc2/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 3fce8a6..1036ea4 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -55,6 +55,7 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlRowOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Stacks;
@@ -160,7 +161,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
             filter.getInput());
       } else if (newConditionExp instanceof RexLiteral
           || RexUtil.isNullLiteral(newConditionExp, true)) {
-        call.transformTo(call.builder().values(filter.getRowType()).build());
+        call.transformTo(createEmptyRelOrEquivalent(call, filter));
       } else if (reduced) {
         call.transformTo(call.builder().
             push(filter.getInput()).filter(expList.get(0)).build());
@@ -182,6 +183,28 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
       call.getPlanner().setImportance(filter, 0.0);
     }
 
+    /**
+     * For static schema systems, a filter that is always false or null can be
+     * replaced by a values operator that produces no rows, as the schema
+     * information can just be taken from the input Rel. In dynamic schema
+     * environments, the filter might have an unknown input type, in these cases
+     * they must define a system specific alternative to a Values operator, such
+     * as inserting a limit 0 instead of a filter on top of the original input.
+     *
+     * <p>The default implementation of this method is to call
+     * {@link RelBuilder#empty}, which for the static schema will be optimized
+     * to an empty
+     * {@link org.apache.calcite.rel.core.Values}.
+     *
+     * @param input rel to replace, assumes caller has already determined
+     *              equivalence to Values operation for 0 records or a
+     *              false filter.
+     * @return equivalent but less expensive replacement rel
+     */
+    protected RelNode createEmptyRelOrEquivalent(RelOptRuleCall call, Filter input) {
+      return call.builder().push(input).empty().build();
+    }
+
     private void reduceNotNullableFilter(
         RelOptRuleCall call,
         Filter filter,
@@ -212,7 +235,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
           if (alwaysTrue) {
             call.transformTo(filter.getInput());
           } else {
-            call.transformTo(call.builder().values(filter.getRowType()).build());
+            call.transformTo(createEmptyRelOrEquivalent(call, filter));
           }
         }
       }
@@ -343,7 +366,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
               || RexUtil.isNullLiteral(newConditionExp, true)) {
             // condition is always NULL or FALSE - replace calc
             // with empty
-            call.transformTo(call.builder().values(calc.getRowType()).build());
+            call.transformTo(createEmptyRelOrEquivalent(call, calc));
             return;
           } else {
             builder.addCondition(list.get(conditionIndex));
@@ -363,6 +386,28 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
         call.getPlanner().setImportance(calc, 0.0);
       }
     }
+
+    /**
+     * For static schema systems, a filter that is always false or null can be
+     * replaced by a values operator that produces no rows, as the schema
+     * information can just be taken from the input Rel. In dynamic schema
+     * environments, the filter might have an unknown input type, in these cases
+     * they must define a system specific alternative to a Values operator, such
+     * as inserting a limit 0 instead of a filter on top of the original input.
+     *
+     * <p>The default implementation of this method is to call
+     * {@link RelBuilder#empty}, which for the static schema will be optimized
+     * to an empty
+     * {@link org.apache.calcite.rel.core.Values}.
+     *
+     * @param input rel to replace, assumes caller has already determined
+     *              equivalence to Values operation for 0 records or a
+     *              false filter.
+     * @return equivalent but less expensive replacement rel
+     */
+    protected RelNode createEmptyRelOrEquivalent(RelOptRuleCall call, Calc input) {
+      return call.builder().push(input).empty().build();
+    }
   }
 
   //~ Constructors -----------------------------------------------------------


[18/50] [abbrv] calcite git commit: [CALCITE-854] Implement UNNEST ... WITH ORDINALITY

Posted by jh...@apache.org.
[CALCITE-854] Implement UNNEST ... WITH ORDINALITY

Also allow UNNEST applied to ARRAY types, and lay groundwork for UNNEST with more than one argument.

Improve the error location for the "List of column aliases must have same degree as table" message.


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

Branch: refs/heads/branch-release
Commit: 4762b889699f5fb963749dca8b3106c17f44f2a9
Parents: 8bc5f85
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Aug 27 16:17:21 2015 -0600
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  11 +-
 .../adapter/enumerable/EnumerableUncollect.java |  81 +++++++++++++--
 .../enumerable/EnumerableUncollectRule.java     |   9 +-
 .../calcite/adapter/enumerable/PhysType.java    |   6 ++
 .../adapter/enumerable/PhysTypeImpl.java        |  22 ++++
 .../calcite/prepare/CalcitePrepareImpl.java     |   5 +-
 .../org/apache/calcite/rel/core/Uncollect.java  | 102 +++++++++++++------
 .../calcite/rel/rules/ProjectMergeRule.java     |   2 +-
 .../apache/calcite/sql/SqlUnnestOperator.java   |  35 ++++++-
 .../sql/fun/SqlArrayQueryConstructor.java       |   2 +-
 .../sql/fun/SqlArrayValueConstructor.java       |   2 +-
 .../sql/fun/SqlMultisetQueryConstructor.java    |   2 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  10 +-
 .../calcite/sql/validate/AbstractNamespace.java |  15 ++-
 .../calcite/sql/validate/AliasNamespace.java    |  12 ++-
 .../calcite/sql/validate/UnnestNamespace.java   |  11 +-
 .../calcite/sql2rel/SqlToRelConverter.java      |  33 +++---
 .../calcite/sql/parser/SqlParserTest.java       |  14 +++
 .../java/org/apache/calcite/test/JdbcTest.java  |  11 +-
 .../calcite/test/SqlToRelConverterTest.java     |  10 ++
 .../apache/calcite/test/SqlValidatorTest.java   |  72 ++++++++++++-
 .../calcite/test/SqlToRelConverterTest.xml      |  28 +++++
 site/_docs/reference.md                         |   2 +-
 23 files changed, 409 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 fe509af..f41814a 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -76,6 +76,7 @@ import org.apache.calcite.sql.SqlSelectKeyword;
 import org.apache.calcite.sql.SqlSetOption;
 import org.apache.calcite.sql.SqlTimeLiteral;
 import org.apache.calcite.sql.SqlTimestampLiteral;
+import org.apache.calcite.sql.SqlUnnestOperator;
 import org.apache.calcite.sql.SqlUpdate;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWindow;
@@ -1675,6 +1676,7 @@ SqlNode TableRef() :
     boolean isRepeatable = false;
     int repeatableSeed = 0;
     SqlNodeList columnAliasList = null;
+    SqlUnnestOperator unnestOp = SqlStdOperatorTable.UNNEST;
 }
 {
     (
@@ -1713,10 +1715,13 @@ SqlNode TableRef() :
     |
         <UNNEST> { pos = getPos(); }
         args = ParenthesizedQueryOrCommaList(ExprContext.ACCEPT_SUBQUERY)
+        [
+            <WITH> <ORDINALITY> {
+                unnestOp = SqlStdOperatorTable.UNNEST_WITH_ORDINALITY;
+            }
+        ]
         {
-            tableRef =
-                SqlStdOperatorTable.UNNEST.createCall(
-                    pos.plus(getPos()), args.toArray());
+            tableRef = unnestOp.createCall(pos.plus(getPos()), args.toArray());
         }
     |
         <TABLE> { pos = getPos(); } <LPAREN>

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java
index 861a4dd..20bbaf2 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollect.java
@@ -16,30 +16,60 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
-import org.apache.calcite.adapter.java.JavaTypeFactory;
 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.ParameterExpression;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Uncollect;
-import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableIntList;
+
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.List;
 
 /** Implementation of {@link org.apache.calcite.rel.core.Uncollect} in
  * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
 public class EnumerableUncollect extends Uncollect implements EnumerableRel {
+  @Deprecated // to be removed before 2.0
   public EnumerableUncollect(RelOptCluster cluster, RelTraitSet traitSet,
       RelNode child) {
-    super(cluster, traitSet, child);
+    this(cluster, traitSet, child, false);
+  }
+
+  /** Creates an EnumerableUncollect.
+   *
+   * <p>Use {@link #create} unless you know what you're doing. */
+  public EnumerableUncollect(RelOptCluster cluster, RelTraitSet traitSet,
+      RelNode child, boolean withOrdinality) {
+    super(cluster, traitSet, child, withOrdinality);
     assert getConvention() instanceof EnumerableConvention;
     assert getConvention() == child.getConvention();
   }
 
+  /**
+   * Creates an EnumerableUncollect.
+   *
+   * <p>Each field of the input relational expression must be an array or
+   * multiset.
+   *
+   * @param traitSet Trait set
+   * @param input    Input relational expression
+   * @param withOrdinality Whether output should contain an ORDINALITY column
+   */
+  public static EnumerableUncollect create(RelTraitSet traitSet, RelNode input,
+      boolean withOrdinality) {
+    final RelOptCluster cluster = input.getCluster();
+    return new EnumerableUncollect(cluster, traitSet, input, withOrdinality);
+  }
+
   @Override public EnumerableUncollect copy(RelTraitSet traitSet,
       RelNode newInput) {
-    return new EnumerableUncollect(getCluster(), traitSet, newInput);
+    return new EnumerableUncollect(getCluster(), traitSet, newInput,
+        withOrdinality);
   }
 
   public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
@@ -52,19 +82,54 @@ public class EnumerableUncollect extends Uncollect implements EnumerableRel {
             getRowType(),
             JavaRowFormat.LIST);
 
-    final JavaTypeFactory typeFactory = implementor.getTypeFactory();
-    RelDataType inputRowType = child.getRowType();
-
     // final Enumerable<List<Employee>> child = <<child adapter>>;
     // return child.selectMany(LIST_TO_ENUMERABLE);
     final Expression child_ =
         builder.append(
             "child", result.block);
+    final Expression lambda;
+    if (withOrdinality) {
+      final BlockBuilder builder2 = new BlockBuilder();
+      final ParameterExpression o_ = Expressions.parameter(Modifier.FINAL,
+          result.physType.getJavaRowType(),
+          "o");
+      final Expression list_ = builder2.append("list",
+          Expressions.new_(ArrayList.class));
+      final ParameterExpression i_ = Expressions.parameter(int.class, "i");
+      final BlockBuilder builder3 = new BlockBuilder();
+      final Expression v_ =
+          builder3.append("v",
+              Expressions.call(o_, BuiltInMethod.LIST_GET.method, i_));
+      final List<Expression> expressions = new ArrayList<>();
+      final PhysType componentPhysType = result.physType.component(0);
+      final int fieldCount = componentPhysType.getRowType().getFieldCount();
+      expressions.addAll(
+          componentPhysType.accessors(v_,
+              ImmutableIntList.identity(fieldCount)));
+      expressions.add(Expressions.add(i_, Expressions.constant(1)));
+      builder3.add(
+          Expressions.statement(
+              Expressions.call(list_, BuiltInMethod.COLLECTION_ADD.method,
+                  physType.record(expressions))));
+      builder2.add(
+          Expressions.for_(
+              Expressions.declare(0, i_, Expressions.constant(0)),
+              Expressions.lessThan(i_,
+                  Expressions.call(o_, BuiltInMethod.COLLECTION_SIZE.method)),
+              Expressions.postIncrementAssign(i_),
+              builder3.toBlock()));
+      builder2.add(
+          Expressions.return_(null,
+              Expressions.call(BuiltInMethod.AS_ENUMERABLE2.method, list_)));
+      lambda = Expressions.lambda(builder2.toBlock(), o_);
+    } else {
+      lambda = Expressions.call(BuiltInMethod.LIST_TO_ENUMERABLE.method);
+    }
     builder.add(
         Expressions.return_(null,
             Expressions.call(child_,
                 BuiltInMethod.SELECT_MANY.method,
-                Expressions.call(BuiltInMethod.LIST_TO_ENUMERABLE.method))));
+                lambda)));
     return implementor.result(physType, builder.toBlock());
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
index 797bbc2..2687b1e 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
@@ -37,11 +37,10 @@ class EnumerableUncollectRule extends ConverterRule {
     final RelTraitSet traitSet =
         uncollect.getTraitSet().replace(EnumerableConvention.INSTANCE);
     final RelNode input = uncollect.getInput();
-    return new EnumerableUncollect(
-        rel.getCluster(),
-        traitSet,
-        convert(input,
-            input.getTraitSet().replace(EnumerableConvention.INSTANCE)));
+    final RelNode newInput = convert(input,
+        input.getTraitSet().replace(EnumerableConvention.INSTANCE));
+    return EnumerableUncollect.create(traitSet, newInput,
+        uncollect.withOrdinality);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
index dd6d2b2..0cf1dfe 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
@@ -48,6 +48,12 @@ public interface PhysType {
    * field type is {@code Object} even if the field is not nullable.</p> */
   Type getJavaFieldType(int field);
 
+  /** Returns the physical type of a field. */
+  PhysType field(int ordinal);
+
+  /** Returns the physical type of a given field's component type. */
+  PhysType component(int field);
+
   /** Returns the SQL row type. */
   RelDataType getRowType();
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
index 918833e..2c0c491 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
@@ -32,6 +32,7 @@ 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.runtime.Utilities;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Pair;
@@ -485,6 +486,27 @@ public class PhysTypeImpl implements PhysType {
     return format.javaFieldClass(typeFactory, rowType, index);
   }
 
+  public PhysType component(int fieldOrdinal) {
+    final RelDataTypeField field = rowType.getFieldList().get(fieldOrdinal);
+    return PhysTypeImpl.of(typeFactory,
+        toStruct(field.getType().getComponentType()), format, false);
+  }
+
+  public PhysType field(int ordinal) {
+    final RelDataTypeField field = rowType.getFieldList().get(ordinal);
+    final RelDataType type = field.getType();
+    return PhysTypeImpl.of(typeFactory, toStruct(type), format, false);
+  }
+
+  private RelDataType toStruct(RelDataType type) {
+    if (type.isStruct()) {
+      return type;
+    }
+    return typeFactory.builder()
+        .add(SqlUtil.deriveAliasFromOrdinal(0), type)
+        .build();
+  }
+
   public Expression comparer() {
     return format.comparer();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 b3eb693..dc17ddd 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -114,6 +114,7 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
@@ -598,7 +599,9 @@ public class CalcitePrepareImpl implements CalcitePrepare {
   private <T> CalciteSignature<T> simplePrepare(Context context, String sql) {
     final JavaTypeFactory typeFactory = context.getTypeFactory();
     final RelDataType x =
-        typeFactory.builder().add("EXPR$0", SqlTypeName.INTEGER).build();
+        typeFactory.builder()
+            .add(SqlUtil.deriveAliasFromOrdinal(0), SqlTypeName.INTEGER)
+            .build();
     @SuppressWarnings("unchecked")
     final List<T> list = (List) ImmutableList.of(1);
     final List<String> origin = null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java b/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java
index 2e76332..2d46929 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java
@@ -21,37 +21,46 @@ import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlUnnestOperator;
 import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 import java.util.List;
 
 /**
- * Relational expression that unnests its input's sole column into a
- * relation.
+ * Relational expression that unnests its input's columns into a relation.
+ *
+ * <p>The input may have multiple columns, but each must be a multiset or
+ * array. If {@code withOrdinality}, the output contains an extra
+ * {@code ORDINALITY} column.
  *
  * <p>Like its inverse operation {@link Collect}, Uncollect is generally
  * invoked in a nested loop, driven by
  * {@link org.apache.calcite.rel.logical.LogicalCorrelate} or similar.
  */
 public class Uncollect extends SingleRel {
+  public final boolean withOrdinality;
+
   //~ Constructors -----------------------------------------------------------
 
-  /**
-   * Creates an Uncollect.
-   *
-   * <p>The row type of the child relational expression must contain precisely
-   * one column, that column must be a multiset of records.
-   *
-   * @param cluster Cluster the relational expression belongs to
-   * @param traitSet Traits
-   * @param child   Child relational expression
-   */
+  @Deprecated // to be removed before 2.0
   public Uncollect(RelOptCluster cluster, RelTraitSet traitSet,
       RelNode child) {
-    super(cluster, traitSet, child);
+    this(cluster, traitSet, child, false);
+  }
+
+  /** Creates an Uncollect.
+   *
+   * <p>Use {@link #create} unless you know what you're doing. */
+  public Uncollect(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
+      boolean withOrdinality) {
+    super(cluster, traitSet, input);
+    this.withOrdinality = withOrdinality;
     assert deriveRowType() != null : "invalid child rowtype";
   }
 
@@ -59,11 +68,33 @@ public class Uncollect extends SingleRel {
    * Creates an Uncollect by parsing serialized output.
    */
   public Uncollect(RelInput input) {
-    this(input.getCluster(), input.getTraitSet(), input.getInput());
+    this(input.getCluster(), input.getTraitSet(), input.getInput(),
+        input.getBoolean("withOrdinality", false));
+  }
+
+  /**
+   * Creates an Uncollect.
+   *
+   * <p>Each field of the input relational expression must be an array or
+   * multiset.
+   *
+   * @param traitSet Trait set
+   * @param input    Input relational expression
+   * @param withOrdinality Whether output should contain an ORDINALITY column
+   */
+  public static Uncollect create(RelTraitSet traitSet, RelNode input,
+      boolean withOrdinality) {
+    final RelOptCluster cluster = input.getCluster();
+    return new Uncollect(cluster, traitSet, input, withOrdinality);
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  @Override public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw)
+        .itemIf("withOrdinality", withOrdinality, withOrdinality);
+  }
+
   @Override public final RelNode copy(RelTraitSet traitSet,
       List<RelNode> inputs) {
     return copy(traitSet, sole(inputs));
@@ -71,35 +102,44 @@ public class Uncollect extends SingleRel {
 
   public RelNode copy(RelTraitSet traitSet, RelNode input) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new Uncollect(getCluster(), traitSet, input);
+    return new Uncollect(getCluster(), traitSet, input, withOrdinality);
   }
 
   protected RelDataType deriveRowType() {
-    return deriveUncollectRowType(getInput());
+    return deriveUncollectRowType(input, withOrdinality);
   }
 
   /**
    * Returns the row type returned by applying the 'UNNEST' operation to a
-   * relational expression. The relational expression must have precisely one
-   * column, whose type must be a multiset of structs. The return type is the
-   * type of that column.
+   * relational expression.
+   *
+   * <p>Each column in the relational expression must be a multiset of structs
+   * or an array. The return type is the type of that column, plus an ORDINALITY
+   * column if {@code withOrdinality}.
    */
-  public static RelDataType deriveUncollectRowType(RelNode rel) {
+  public static RelDataType deriveUncollectRowType(RelNode rel,
+      boolean withOrdinality) {
     RelDataType inputType = rel.getRowType();
     assert inputType.isStruct() : inputType + " is not a struct";
     final List<RelDataTypeField> fields = inputType.getFieldList();
-    assert 1 == fields.size() : "expected 1 field";
-    RelDataType ret = fields.get(0).getType().getComponentType();
-    assert null != ret;
-    if (!ret.isStruct()) {
-      // Element type is not a record. It may be a scalar type, say
-      // "INTEGER". Wrap it in a struct type.
-      ret =
-          rel.getCluster().getTypeFactory().builder()
-              .add(SqlUtil.deriveAliasFromOrdinal(0), ret)
-              .build();
+    final RelDataTypeFactory.FieldInfoBuilder builder =
+        rel.getCluster().getTypeFactory().builder();
+    for (RelDataTypeField field : fields) {
+      RelDataType ret = field.getType().getComponentType();
+      assert null != ret;
+      if (ret.isStruct()) {
+        builder.addAll(ret.getFieldList());
+      } else {
+        // Element type is not a record. It may be a scalar type, say
+        // "INTEGER". Wrap it in a struct type.
+        builder.add(SqlUtil.deriveAliasFromOrdinal(field.getIndex()), ret);
+      }
+    }
+    if (withOrdinality) {
+      builder.add(SqlUnnestOperator.ORDINALITY_COLUMN_NAME,
+          SqlTypeName.INTEGER);
     }
-    return ret;
+    return builder.build();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
index 9a4e849..1a67787 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectMergeRule.java
@@ -24,9 +24,9 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.util.Permutation;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java
index 54cbb99..096f361 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUnnestOperator.java
@@ -17,17 +17,26 @@
 package org.apache.calcite.sql;
 
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.ArraySqlType;
 import org.apache.calcite.sql.type.MultisetSqlType;
 import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 /**
  * The <code>UNNEST</code> operator.
  */
 public class SqlUnnestOperator extends SqlFunctionalOperator {
+  /** Whether {@code WITH ORDINALITY} was specified.
+   *
+   * <p>If so, the returned records include a column {@code ORDINALITY}. */
+  public final boolean withOrdinality;
+
+  public static final String ORDINALITY_COLUMN_NAME = "ORDINALITY";
+
   //~ Constructors -----------------------------------------------------------
 
-  public SqlUnnestOperator() {
+  public SqlUnnestOperator(boolean withOrdinality) {
     super(
         "UNNEST",
         SqlKind.UNNEST,
@@ -36,6 +45,7 @@ public class SqlUnnestOperator extends SqlFunctionalOperator {
         null,
         null,
         OperandTypes.SCALAR_OR_RECORD_COLLECTION);
+    this.withOrdinality = withOrdinality;
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -47,7 +57,28 @@ public class SqlUnnestOperator extends SqlFunctionalOperator {
       type = type.getFieldList().get(0).getType();
     }
     assert type instanceof ArraySqlType || type instanceof MultisetSqlType;
-    return type.getComponentType();
+    if (withOrdinality) {
+      final RelDataTypeFactory.FieldInfoBuilder builder =
+          opBinding.getTypeFactory().builder();
+      if (type.getComponentType().isStruct()) {
+        builder.addAll(type.getComponentType().getFieldList());
+      } else {
+        builder.add(SqlUtil.deriveAliasFromOrdinal(0), type.getComponentType());
+      }
+      return builder
+          .add(ORDINALITY_COLUMN_NAME, SqlTypeName.INTEGER)
+          .build();
+    } else {
+      return type.getComponentType();
+    }
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
+      int rightPrec) {
+    super.unparse(writer, call, leftPrec, rightPrec);
+    if (withOrdinality) {
+      writer.keyword("WITH ORDINALITY");
+    }
   }
 
   public boolean argumentMustBeScalar(int ordinal) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java
index 2ea6b0a..640f742 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayQueryConstructor.java
@@ -26,7 +26,7 @@ public class SqlArrayQueryConstructor extends SqlMultisetQueryConstructor {
   //~ Constructors -----------------------------------------------------------
 
   public SqlArrayQueryConstructor() {
-    super("ARRAY", SqlKind.MAP_QUERY_CONSTRUCTOR);
+    super("ARRAY", SqlKind.ARRAY_QUERY_CONSTRUCTOR);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java
index b51c18b..5fc55fe 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlArrayValueConstructor.java
@@ -22,7 +22,7 @@ import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 
 /**
- * Definition of the SQL:2003 standard ARRAY constructor, <code>MULTISET
+ * Definition of the SQL:2003 standard ARRAY constructor, <code>ARRAY
  * [&lt;expr&gt;, ...]</code>.
  */
 public class SqlArrayValueConstructor extends SqlMultisetValueConstructor {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java
index 76b09fe..9c58a30 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMultisetQueryConstructor.java
@@ -122,7 +122,7 @@ public class SqlMultisetQueryConstructor extends SqlSpecialOperator {
       SqlCall call,
       int leftPrec,
       int rightPrec) {
-    writer.keyword("MULTISET");
+    writer.keyword(getName());
     final SqlWriter.Frame frame = writer.startList("(", ")");
     assert call.operandCount() == 1;
     call.operand(0).unparse(writer, leftPrec, rightPrec);

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 ec26b7c..6aa6306 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
@@ -949,8 +949,14 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   /**
    * The <code>UNNEST</code> operator.
    */
-  public static final SqlSpecialOperator UNNEST =
-      new SqlUnnestOperator();
+  public static final SqlUnnestOperator UNNEST =
+      new SqlUnnestOperator(false);
+
+  /**
+   * The <code>UNNEST WITH ORDINALITY</code> operator.
+   */
+  public static final SqlUnnestOperator UNNEST_WITH_ORDINALITY =
+      new SqlUnnestOperator(true);
 
   /**
    * The <code>LATERAL</code> operator.

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
index 9bbb11f..c0c1971 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
@@ -202,10 +202,7 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
       return type;
     }
     final RelDataTypeFactory typeFactory = validator.getTypeFactory();
-    final RelDataType structType =
-        typeFactory.builder()
-            .add(validator.deriveAlias(getNode(), 0), componentType)
-            .build();
+    final RelDataType structType = toStruct(componentType, getNode());
     final RelDataType collectionType;
     switch (type.getSqlTypeName()) {
     case ARRAY:
@@ -220,6 +217,16 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
     return typeFactory.createTypeWithNullability(collectionType,
         type.isNullable());
   }
+
+  /** Converts a type to a struct if it is not already. */
+  protected RelDataType toStruct(RelDataType type, SqlNode unnest) {
+    if (type.isStruct()) {
+      return type;
+    }
+    return validator.getTypeFactory().builder()
+        .add(validator.deriveAlias(unnest, 0), type)
+        .build();
+  }
 }
 
 // End AbstractNamespace.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java
index c109dcf..d425ceb 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AliasNamespace.java
@@ -21,7 +21,9 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.Util;
 
 import java.util.ArrayList;
@@ -67,7 +69,8 @@ public class AliasNamespace extends AbstractNamespace {
     final SqlValidatorNamespace childNs =
         validator.getNamespace(operands.get(0));
     final RelDataType rowType = childNs.getRowTypeSansSystemColumns();
-    for (final SqlNode operand : Util.skip(operands, 2)) {
+    final List<SqlNode> columnNames = Util.skip(operands, 2);
+    for (final SqlNode operand : columnNames) {
       String name = ((SqlIdentifier) operand).getSimple();
       if (nameList.contains(name)) {
         throw validator.newValidationError(operand,
@@ -76,8 +79,11 @@ public class AliasNamespace extends AbstractNamespace {
       nameList.add(name);
     }
     if (nameList.size() != rowType.getFieldCount()) {
-      // Position error at first name in list.
-      throw validator.newValidationError(operands.get(2),
+      // Position error over all column names
+      final SqlNode node = operands.size() == 3
+          ? operands.get(2)
+          : new SqlNodeList(columnNames, SqlParserPos.sum(columnNames));
+      throw validator.newValidationError(node,
           RESOURCE.aliasListDegree(rowType.getFieldCount(), getString(rowType),
               nameList.size()));
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java
index f5829f3..68c1108 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/UnnestNamespace.java
@@ -19,7 +19,7 @@ package org.apache.calcite.sql.validate;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.SqlUnnestOperator;
 import org.apache.calcite.sql.type.MultisetSqlType;
 
 /**
@@ -40,7 +40,7 @@ class UnnestNamespace extends AbstractNamespace {
       SqlNode enclosingNode) {
     super(validator, enclosingNode);
     assert scope != null;
-    assert unnest.getOperator() == SqlStdOperatorTable.UNNEST;
+    assert unnest.getOperator() instanceof SqlUnnestOperator;
     this.unnest = unnest;
     this.scope = scope;
   }
@@ -53,12 +53,7 @@ class UnnestNamespace extends AbstractNamespace {
     RelDataType type =
         unnest.getOperator().validateOperands(validator, scope, unnest);
 
-    if (type.isStruct()) {
-      return type;
-    }
-    return validator.getTypeFactory().builder()
-        .add(validator.deriveAlias(unnest, 0), type)
-        .build();
+    return toStruct(type, unnest);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 e4e5edf..fe40346 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -112,6 +112,7 @@ import org.apache.calcite.sql.SqlSampleSpec;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlSelectKeyword;
 import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.SqlUnnestOperator;
 import org.apache.calcite.sql.SqlUpdate;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlValuesOperator;
@@ -1013,6 +1014,7 @@ public class SqlToRelConverter {
 
     case MULTISET_QUERY_CONSTRUCTOR:
     case MULTISET_VALUE_CONSTRUCTOR:
+    case ARRAY_QUERY_CONSTRUCTOR:
       rel = convertMultisets(ImmutableList.of(subQuery.node), bb);
       subQuery.expr = bb.register(rel, JoinRelType.INNER);
       return;
@@ -1685,6 +1687,7 @@ public class SqlToRelConverter {
     case SELECT:
     case MULTISET_QUERY_CONSTRUCTOR:
     case MULTISET_VALUE_CONSTRUCTOR:
+    case ARRAY_QUERY_CONSTRUCTOR:
     case CURSOR:
     case SCALAR_QUERY:
       if (!registerOnlyScalarSubqueries
@@ -1886,7 +1889,7 @@ public class SqlToRelConverter {
   protected void convertFrom(
       Blackboard bb,
       SqlNode from) {
-    SqlCall call;
+    final SqlCall call;
     final SqlNode[] operands;
     switch (from.getKind()) {
     case AS:
@@ -2026,7 +2029,9 @@ public class SqlToRelConverter {
       return;
 
     case UNNEST:
-      final SqlNode node = ((SqlCall) from).operand(0);
+      call = (SqlCall) from;
+      final SqlNode node = call.operand(0);
+      final SqlUnnestOperator operator = (SqlUnnestOperator) call.getOperator();
       replaceSubqueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
       final RelNode childRel =
           RelOptUtil.createProject(
@@ -2037,7 +2042,7 @@ public class SqlToRelConverter {
 
       Uncollect uncollect =
           new Uncollect(cluster, cluster.traitSetOf(Convention.NONE),
-              childRel);
+              childRel, operator.withOrdinality);
       bb.setRoot(uncollect, true);
       return;
 
@@ -2046,8 +2051,8 @@ public class SqlToRelConverter {
 
       // Dig out real call; TABLE() wrapper is just syntactic.
       assert call.getOperandList().size() == 1;
-      call = call.operand(0);
-      convertCollectionTable(bb, call);
+      final SqlCall call2 = call.operand(0);
+      convertCollectionTable(bb, call2);
       return;
 
     default:
@@ -3348,14 +3353,10 @@ public class SqlToRelConverter {
       }
 
       final SqlCall call = (SqlCall) operand;
-      final SqlOperator op = call.getOperator();
-      if ((op != SqlStdOperatorTable.MULTISET_VALUE)
-          && (op != SqlStdOperatorTable.MULTISET_QUERY)) {
-        lastList.add(operand);
-        continue;
-      }
       final RelNode input;
-      if (op == SqlStdOperatorTable.MULTISET_VALUE) {
+      switch (call.getKind()) {
+      case MULTISET_VALUE_CONSTRUCTOR:
+      case ARRAY_VALUE_CONSTRUCTOR:
         final SqlNodeList list =
             new SqlNodeList(call.getOperandList(), call.getParserPosition());
         CollectNamespace nss =
@@ -3376,9 +3377,15 @@ public class SqlToRelConverter {
             list,
             multisetType.getComponentType());
         input = convertQueryOrInList(usedBb, list, null);
-      } else {
+        break;
+      case MULTISET_QUERY_CONSTRUCTOR:
+      case ARRAY_QUERY_CONSTRUCTOR:
         final RelRoot root = convertQuery(call.operand(0), false, true);
         input = root.rel;
+        break;
+      default:
+        lastList.add(operand);
+        continue;
       }
 
       if (lastList.size() > 0) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 c1c8f3b..31dbd58 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
@@ -5541,6 +5541,20 @@ public class SqlParserTest {
         "(?s)Encountered \"unnest\" at.*");
   }
 
+  @Test public void testUnnestWithOrdinality() {
+    sql("select * from unnest(x) with ordinality")
+        .ok("SELECT *\n"
+            + "FROM (UNNEST(`X`) WITH ORDINALITY)");
+    sql("select*from unnest(x) with ordinality AS T")
+        .ok("SELECT *\n"
+            + "FROM (UNNEST(`X`) WITH ORDINALITY) AS `T`");
+    sql("select*from unnest(x) with ordinality AS T(c, o)")
+        .ok("SELECT *\n"
+            + "FROM (UNNEST(`X`) WITH ORDINALITY) AS `T` (`C`, `O`)");
+    sql("select*from unnest(x) as T ^with^ ordinality")
+        .fails("(?s)Encountered \"with\" at .*");
+  }
+
   @Test public void testParensInFrom() {
     // UNNEST may not occur within parentheses.
     // FIXME should fail at "unnest"

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 3ee40e8..b64cba4 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -2176,11 +2176,18 @@ public class JdbcTest {
         .returnsUnordered("A=[{10}, {20}, {10}, {10}]");
   }
 
-  @Ignore("unnest does not apply to array. should it?")
   @Test public void testUnnestArray() {
     CalciteAssert.that()
         .query("select*from unnest(array[1,2])")
-        .returnsUnordered("xx");
+        .returnsUnordered("EXPR$0=1",
+            "EXPR$0=2");
+  }
+
+  @Test public void testUnnestArrayWithOrdinality() {
+    CalciteAssert.that()
+        .query("select*from unnest(array[10,20]) with ordinality as t(i, o)")
+        .returnsUnordered("I=10; O=1",
+            "I=20; O=2");
   }
 
   @Test public void testUnnestMultiset() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 1c67bc8..c24e6af 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -734,6 +734,16 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     check("select*from unnest(multiset(select*from dept))", "${plan}");
   }
 
+  @Test public void testUnnestArray() {
+    sql("select*from unnest(array(select*from dept))")
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testUnnestWithOrdinality() {
+    sql("select*from unnest(array(select*from dept)) with ordinality")
+        .convertsTo("${plan}");
+  }
+
   @Test public void testMultisetSubquery() {
     check(
         "select multiset(select deptno from dept) from (values(true))",

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 64d1c56..2f4f60e 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -4606,7 +4606,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkFails(
         "select d.^deptno^ from dept as d(a, b)",
         "(?s).*Column 'DEPTNO' not found in table 'D'.*");
-    checkFails("select 1 from dept as d(^a^, b, c)",
+    checkFails("select 1 from dept as d(^a, b, c^)",
         "(?s).*List of column aliases must have same degree as table; "
             + "table has 2 columns \\('DEPTNO', 'NAME'\\), "
             + "whereas alias list has 3 columns.*");
@@ -6501,6 +6501,76 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Column 'C1' not found in any table");
   }
 
+  @Test public void testUnnestArray() {
+    checkColumnType("select*from unnest(array[1])", "INTEGER NOT NULL");
+    checkColumnType("select*from unnest(array[1, 2])", "INTEGER NOT NULL");
+    checkColumnType(
+        "select*from unnest(array[321.3, 2.33])",
+        "DECIMAL(5, 2) NOT NULL");
+    checkColumnType(
+        "select*from unnest(array[321.3, 4.23e0])",
+        "DOUBLE NOT NULL");
+    checkColumnType(
+        "select*from unnest(array[43.2e1, cast(null as decimal(4,2))])",
+        "DOUBLE");
+    checkColumnType(
+        "select*from unnest(array[1, 2.3, 1])",
+        "DECIMAL(11, 1) NOT NULL");
+    checkColumnType(
+        "select*from unnest(array['1','22','333'])",
+        "CHAR(3) NOT NULL");
+    checkColumnType(
+        "select*from unnest(array['1','22','333','22'])",
+        "CHAR(3) NOT NULL");
+    checkFails(
+        "select*from ^unnest(1)^",
+        "(?s).*Cannot apply 'UNNEST' to arguments of type 'UNNEST.<INTEGER>.'.*");
+    check("select*from unnest(array(select*from dept))");
+    check("select c from unnest(array(select deptno from dept)) as t(c)");
+    checkFails("select c from unnest(array(select * from dept)) as t(^c^)",
+        "List of column aliases must have same degree as table; table has 2 columns \\('DEPTNO', 'NAME'\\), whereas alias list has 1 columns");
+    checkFails(
+        "select ^c1^ from unnest(array(select name from dept)) as t(c)",
+        "Column 'C1' not found in any table");
+  }
+
+  @Test public void testUnnestWithOrdinality() {
+    checkResultType("select*from unnest(array[1, 2]) with ordinality",
+        "RecordType(INTEGER NOT NULL EXPR$0, INTEGER NOT NULL ORDINALITY) NOT NULL");
+    checkResultType(
+        "select*from unnest(array[43.2e1, cast(null as decimal(4,2))]) with ordinality",
+        "RecordType(DOUBLE EXPR$0, INTEGER NOT NULL ORDINALITY) NOT NULL");
+    checkFails(
+        "select*from ^unnest(1) with ordinality^",
+        "(?s).*Cannot apply 'UNNEST' to arguments of type 'UNNEST.<INTEGER>.'.*");
+    check("select deptno\n"
+        + "from unnest(array(select*from dept)) with ordinality\n"
+        + "where ordinality < 5");
+    checkFails("select c from unnest(\n"
+        + "  array(select deptno from dept)) with ordinality as t(^c^)",
+        "List of column aliases must have same degree as table; table has 2 "
+        + "columns \\('DEPTNO', 'ORDINALITY'\\), "
+        + "whereas alias list has 1 columns");
+    check("select c from unnest(\n"
+        + "  array(select deptno from dept)) with ordinality as t(c, d)");
+    checkFails("select c from unnest(\n"
+        + "  array(select deptno from dept)) with ordinality as t(^c, d, e^)",
+        "List of column aliases must have same degree as table; table has 2 "
+        + "columns \\('DEPTNO', 'ORDINALITY'\\), "
+        + "whereas alias list has 3 columns");
+    checkFails("select c\n"
+        + "from unnest(array(select * from dept)) with ordinality as t(^c, d, e, f^)",
+        "List of column aliases must have same degree as table; table has 3 "
+        + "columns \\('DEPTNO', 'NAME', 'ORDINALITY'\\), "
+        + "whereas alias list has 4 columns");
+    checkFails(
+        "select ^name^ from unnest(array(select name from dept)) with ordinality as t(c, o)",
+        "Column 'NAME' not found in any table");
+    checkFails(
+        "select ^ordinality^ from unnest(array(select name from dept)) with ordinality as t(c, o)",
+        "Column 'ORDINALITY' not found in any table");
+  }
+
   @Test public void testCorrelationJoin() {
     check("select *,"
         + "         multiset(select * from emp where deptno=dept.deptno) "

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/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 c46d493..553f275 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -145,6 +145,34 @@ LogicalProject(DEPTNO=[$0], NAME=[$1])
             <![CDATA[select*from unnest(multiset(select*from dept))]]>
         </Resource>
     </TestCase>
+    <TestCase name="testUnnestArray">
+        <Resource name="sql">
+            <![CDATA[select*from unnest(array(select*from dept))]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0], NAME=[$1])
+  Uncollect
+    Collect(field=[EXPR$0])
+      LogicalProject(DEPTNO=[$0], NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testUnnestWithOrdinality">
+        <Resource name="sql">
+            <![CDATA[select*from unnest(array(select*from dept)) with ordinality]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0], NAME=[$1], ORDINALITY=[$2])
+  Uncollect(withOrdinality=[true])
+    Collect(field=[EXPR$0])
+      LogicalProject(DEPTNO=[$0], NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testMultisetSubquery">
         <Resource name="plan">
             <![CDATA[

http://git-wip-us.apache.org/repos/asf/calcite/blob/4762b889/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 7f267bd..dbf2974 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -128,7 +128,7 @@ tablePrimary:
       [ TABLE ] [ [ catalogName . ] schemaName . ] tableName
   |   '(' query ')'
   |   values
-  |   UNNEST '(' expression ')'
+  |   UNNEST '(' expression ')' [ WITH ORDINALITY ]
   |   TABLE '(' [ SPECIFIC ] functionName '(' expression [, expression ]* ')' ')'
 
 values:


[27/50] [abbrv] calcite git commit: Tune algorithm that deduces the return type of AND expression

Posted by jh...@apache.org.
Tune algorithm that deduces the return type of AND expression


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

Branch: refs/heads/branch-release
Commit: 2cef85941359c530e7f0ed40265ce3c6f9db6bf8
Parents: 4b519b9
Author: Julian Hyde <jh...@apache.org>
Authored: Sat Jul 11 08:18:13 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:25 2016 -0800

----------------------------------------------------------------------
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  6 ++---
 .../apache/calcite/sql/type/ReturnTypes.java    | 23 +++++++++++++++++---
 2 files changed, 23 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/2cef8594/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 6aa6306..464945d 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
@@ -138,7 +138,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           SqlKind.AND,
           28,
           true,
-          ReturnTypes.BOOLEAN_NULLABLE,
+          ReturnTypes.ARG0_NULLABLE, // more efficient than BOOLEAN_NULLABLE
           InferTypes.BOOLEAN,
           OperandTypes.BOOLEAN_BOOLEAN);
 
@@ -427,7 +427,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           SqlKind.OR,
           26,
           true,
-          ReturnTypes.BOOLEAN_NULLABLE,
+          ReturnTypes.ARG0_NULLABLE, // more efficient than BOOLEAN_NULLABLE
           InferTypes.BOOLEAN,
           OperandTypes.BOOLEAN_BOOLEAN);
 
@@ -628,7 +628,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           "NOT",
           SqlKind.NOT,
           30,
-          ReturnTypes.BOOLEAN_NULLABLE,
+          ReturnTypes.ARG0,
           InferTypes.BOOLEAN,
           OperandTypes.BOOLEAN);
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/2cef8594/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 fa99579..a81caf3 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
@@ -95,16 +95,33 @@ public abstract class ReturnTypes {
    * returned type will also be nullable. First Arg must be of string type.
    */
   public static final SqlReturnTypeInference ARG0_NULLABLE_VARYING =
-      cascade(
-          ARG0, SqlTypeTransforms.TO_NULLABLE,
+      cascade(ARG0, SqlTypeTransforms.TO_NULLABLE,
           SqlTypeTransforms.TO_VARYING);
+
   /**
    * Type-inference strategy whereby the result type of a call is the type of
    * the operand #0 (0-based). If any of the other operands are nullable the
    * returned type will also be nullable.
    */
   public static final SqlReturnTypeInference ARG0_NULLABLE =
-      cascade(ARG0, SqlTypeTransforms.TO_NULLABLE);
+      new SqlReturnTypeInference() {
+        // Equivalent to
+        //   cascade(ARG0, SqlTypeTransforms.TO_NULLABLE);
+        // but implemented by hand because used in AND, which is a very common
+        // operator.
+        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+          final int n = opBinding.getOperandCount();
+          RelDataType type1 = null;
+          for (int i = 0; i < n; i++) {
+            type1 = opBinding.getOperandType(i);
+            if (type1.isNullable()) {
+              break;
+            }
+          }
+          return type1;
+        }
+      };
+
   /**
    * Type-inference strategy whereby the result type of a call is the type of
    * the operand #0 (0-based), with nulls always allowed.


[23/50] [abbrv] calcite git commit: [CALCITE-816] Represent sub-query as a RexNode

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 5c82a96..2294c4a 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -45,6 +45,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPermuteInputsShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.Bug;
@@ -68,6 +69,7 @@ import java.util.Collections;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.logging.Level;
 
 /**
  * Transformer that walks over a tree of relational expressions, replacing each
@@ -156,6 +158,11 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     if (!trimResult.right.isIdentity()) {
       throw new IllegalArgumentException();
     }
+    if (SqlToRelConverter.SQL2REL_LOGGER.isLoggable(Level.FINE)) {
+      SqlToRelConverter.SQL2REL_LOGGER.fine(
+          RelOptUtil.dumpPlan("Plan after trimming unused fields",
+              trimResult.left, false, SqlExplainLevel.EXPPLAN_ATTRIBUTES));
+    }
     return trimResult.left;
   }
 
@@ -434,8 +441,9 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     RexNode newConditionExpr =
         conditionExpr.accept(shuttle);
 
-    relBuilder.push(newInput);
-    relBuilder.filter(newConditionExpr);
+    // Use copy rather than relBuilder so that correlating variables get set.
+    relBuilder.push(
+        filter.copy(filter.getTraitSet(), newInput, newConditionExpr));
 
     // The result has the same mapping as the input gave us. Sometimes we
     // return fields that the consumer didn't ask for, because the filter

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
index 4d8f0a5..e7278d6 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
@@ -390,7 +390,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
 
   public void rewriteRel(LogicalFilter rel) {
     RelNode newRel =
-        RelOptUtil.createFilter(
+        rel.copy(rel.getTraitSet(),
             getNewForOldRel(rel.getInput()),
             rel.getCondition().accept(new RewriteRexShuttle()));
     setNewForOldRel(rel, newRel);
@@ -401,14 +401,13 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         LogicalJoin.create(getNewForOldRel(rel.getLeft()),
             getNewForOldRel(rel.getRight()),
             rel.getCondition().accept(new RewriteRexShuttle()),
-            rel.getJoinType(),
-            rel.getVariablesStopped());
+            rel.getVariablesSet(), rel.getJoinType());
     setNewForOldRel(rel, newRel);
   }
 
   public void rewriteRel(LogicalCorrelate rel) {
     ImmutableBitSet.Builder newPos = ImmutableBitSet.builder();
-    for (Integer pos : rel.getRequiredColumns()) {
+    for (int pos : rel.getRequiredColumns()) {
       RelDataType corrFieldType =
           rel.getLeft().getRowType().getFieldList().get(pos)
               .getType();

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 fe40346..31e8f88 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -33,11 +33,12 @@ import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
-import org.apache.calcite.rel.RelShuttle;
+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;
+import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -48,6 +49,7 @@ import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.Uncollect;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalIntersect;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalMinus;
@@ -76,8 +78,8 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexRangeRef;
 import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.schema.ModifiableView;
@@ -227,11 +229,8 @@ public class SqlToRelConverter {
   /**
    * Fields used in name resolution for correlated subqueries.
    */
-  private final Map<String, DeferredLookup> mapCorrelToDeferred =
+  private final Map<CorrelationId, DeferredLookup> mapCorrelToDeferred =
       new HashMap<>();
-  private int nextCorrel = 0;
-
-  private static final String CORREL_PREFIX = "$cor";
 
   /**
    * Stack of names of datasets requested by the <code>
@@ -249,6 +248,10 @@ public class SqlToRelConverter {
 
   public final RelOptTable.ViewExpander viewExpander;
 
+  /** Whether to expand sub-queries. If false, each sub-query becomes a
+   * {@link org.apache.calcite.rex.RexSubQuery}. */
+  private boolean expand = true;
+
   //~ Constructors -----------------------------------------------------------
   /**
    * Creates a converter.
@@ -979,11 +982,23 @@ public class SqlToRelConverter {
     final RexNode convertedWhere = bb.convertExpression(newWhere);
 
     // only allocate filter if the condition is not TRUE
-    if (!convertedWhere.isAlwaysTrue()) {
-      bb.setRoot(
-          RelOptUtil.createFilter(bb.root, convertedWhere),
-          false);
+    if (convertedWhere.isAlwaysTrue()) {
+      return;
+    }
+
+    final RelNode filter = RelOptUtil.createFilter(bb.root, convertedWhere);
+    final RelNode r;
+    final CorrelationUse p = getCorrelationUse(bb, filter);
+    if (p != null) {
+      assert p.r instanceof Filter;
+      Filter f = (Filter) p.r;
+      r = LogicalFilter.create(f.getInput(), f.getCondition(),
+          ImmutableSet.of(p.id));
+    } else {
+      r = filter;
     }
+
+    bb.setRoot(r, false);
   }
 
   private void replaceSubqueries(
@@ -1021,10 +1036,11 @@ public class SqlToRelConverter {
 
     case IN:
       call = (SqlBasicCall) subQuery.node;
-      final SqlNode[] operands = call.getOperands();
-
-      SqlNode leftKeyNode = operands[0];
-      query = operands[1];
+      query = call.operand(1);
+      if (!expand && !(query instanceof SqlNodeList)) {
+        return;
+      }
+      final SqlNode leftKeyNode = call.operand(0);
 
       final List<RexNode> leftKeys;
       switch (leftKeyNode.getKind()) {
@@ -1106,11 +1122,8 @@ public class SqlToRelConverter {
                     AggregateCall.create(SqlStdOperatorTable.COUNT, false,
                         args, -1, longType, null)));
         LogicalJoin join =
-            LogicalJoin.create(bb.root,
-                aggregate,
-                rexBuilder.makeLiteral(true),
-                JoinRelType.INNER,
-                ImmutableSet.<String>of());
+            LogicalJoin.create(bb.root, aggregate, rexBuilder.makeLiteral(true),
+                ImmutableSet.<CorrelationId>of(), JoinRelType.INNER);
         bb.setRoot(join, false);
       }
       RexNode rex =
@@ -1135,7 +1148,10 @@ public class SqlToRelConverter {
       // If there is no correlation, the expression is replaced with a
       // boolean indicating whether the subquery returned 0 or >= 1 row.
       call = (SqlBasicCall) subQuery.node;
-      query = call.getOperands()[0];
+      query = call.operand(0);
+      if (!expand) {
+        return;
+      }
       converted = convertExists(query, RelOptUtil.SubqueryType.EXISTS,
           subQuery.logic, true, null);
       assert !converted.right;
@@ -1148,8 +1164,11 @@ public class SqlToRelConverter {
     case SCALAR_QUERY:
       // Convert the subquery.  If it's non-correlated, convert it
       // to a constant expression.
+      if (!expand) {
+        return;
+      }
       call = (SqlBasicCall) subQuery.node;
-      query = call.getOperands()[0];
+      query = call.operand(0);
       converted = convertExists(query, RelOptUtil.SubqueryType.SCALAR,
           subQuery.logic, true, null);
       assert !converted.right;
@@ -1401,7 +1420,7 @@ public class SqlToRelConverter {
         assert rightVals instanceof SqlCall;
         final SqlBasicCall call = (SqlBasicCall) rightVals;
         assert (call.getOperator() instanceof SqlRowOperator)
-            && call.getOperands().length == leftKeys.size();
+            && call.operandCount() == leftKeys.size();
         rexComparison =
             RexUtil.composeConjunction(
                 rexBuilder,
@@ -1893,8 +1912,7 @@ public class SqlToRelConverter {
     final SqlNode[] operands;
     switch (from.getKind()) {
     case AS:
-      operands = ((SqlBasicCall) from).getOperands();
-      convertFrom(bb, operands[0]);
+      convertFrom(bb, ((SqlCall) from).operand(0));
       return;
 
     case WITH_ITEM:
@@ -2145,140 +2163,132 @@ public class SqlToRelConverter {
       JoinRelType joinType) {
     assert joinCond != null;
 
-    Set<String> correlatedVariables = RelOptUtil.getVariablesUsed(rightRel);
-    if (correlatedVariables.size() > 0) {
-      final ImmutableBitSet.Builder requiredColumns = ImmutableBitSet.builder();
-      final List<String> correlNames = Lists.newArrayList();
+    final CorrelationUse p = getCorrelationUse(bb, rightRel);
+    if (p != null) {
+      LogicalCorrelate corr = LogicalCorrelate.create(leftRel, p.r,
+          p.id, p.requiredColumns, SemiJoinType.of(joinType));
+      if (!joinCond.isAlwaysTrue()) {
+        return RelOptUtil.createFilter(corr, joinCond);
+      }
+      return corr;
+    }
 
-      // All correlations must refer the same namespace since correlation
-      // produces exactly one correlation source.
-      // The same source might be referenced by different variables since
-      // DeferredLookups are not de-duplicated at create time.
-      SqlValidatorNamespace prevNs = null;
+    final Join originalJoin =
+        (Join) RelFactories.DEFAULT_JOIN_FACTORY.createJoin(leftRel, rightRel,
+            joinCond, ImmutableSet.<CorrelationId>of(), joinType, false);
 
-      for (String correlName : correlatedVariables) {
-        DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
-        RexFieldAccess fieldAccess = lookup.getFieldAccess(correlName);
-        String originalRelName = lookup.getOriginalRelName();
-        String originalFieldName = fieldAccess.getField().getName();
+    return RelOptUtil.pushDownJoinConditions(originalJoin);
+  }
 
-        int[] nsIndexes = {-1};
-        final SqlValidatorScope[] ancestorScopes = {null};
-        SqlValidatorNamespace foundNs =
-            lookup.bb.scope.resolve(
-                ImmutableList.of(originalRelName),
-                ancestorScopes,
-                nsIndexes);
+  private CorrelationUse getCorrelationUse(Blackboard bb, final RelNode r0) {
+    final Set<CorrelationId> correlatedVariables =
+        RelOptUtil.getVariablesUsed(r0);
+    if (correlatedVariables.isEmpty()) {
+      return null;
+    }
+    final ImmutableBitSet.Builder requiredColumns = ImmutableBitSet.builder();
+    final List<CorrelationId> correlNames = Lists.newArrayList();
 
-        assert foundNs != null;
-        assert nsIndexes.length == 1;
+    // All correlations must refer the same namespace since correlation
+    // produces exactly one correlation source.
+    // The same source might be referenced by different variables since
+    // DeferredLookups are not de-duplicated at create time.
+    SqlValidatorNamespace prevNs = null;
 
-        int childNamespaceIndex = nsIndexes[0];
+    for (CorrelationId correlName : correlatedVariables) {
+      DeferredLookup lookup =
+          mapCorrelToDeferred.get(correlName);
+      RexFieldAccess fieldAccess = lookup.getFieldAccess(correlName);
+      String originalRelName = lookup.getOriginalRelName();
+      String originalFieldName = fieldAccess.getField().getName();
 
-        SqlValidatorScope ancestorScope = ancestorScopes[0];
-        boolean correlInCurrentScope = ancestorScope == bb.scope;
+      int[] nsIndexes = {-1};
+      final SqlValidatorScope[] ancestorScopes = {null};
+      SqlValidatorNamespace foundNs =
+          lookup.bb.scope.resolve(
+              ImmutableList.of(originalRelName),
+              ancestorScopes,
+              nsIndexes);
 
-        if (!correlInCurrentScope) {
-          continue;
-        }
+      assert foundNs != null;
+      assert nsIndexes.length == 1;
 
-        if (prevNs == null) {
-          prevNs = foundNs;
-        } else {
-          assert prevNs == foundNs : "All correlation variables should resolve"
-              + " to the same namespace."
-              + " Prev ns=" + prevNs
-              + ", new ns=" + foundNs;
-        }
+      int childNamespaceIndex = nsIndexes[0];
 
-        int namespaceOffset = 0;
-        if (childNamespaceIndex > 0) {
-          // If not the first child, need to figure out the width
-          // of output types from all the preceding namespaces
-          assert ancestorScope instanceof ListScope;
-          List<SqlValidatorNamespace> children =
-              ((ListScope) ancestorScope).getChildren();
-
-          for (int i = 0; i < childNamespaceIndex; i++) {
-            SqlValidatorNamespace child = children.get(i);
-            namespaceOffset +=
-                child.getRowType().getFieldCount();
-          }
-        }
+      SqlValidatorScope ancestorScope = ancestorScopes[0];
+      boolean correlInCurrentScope = ancestorScope == bb.scope;
 
-        RelDataTypeField field =
-            catalogReader.field(foundNs.getRowType(), originalFieldName);
-        int pos = namespaceOffset + field.getIndex();
+      if (!correlInCurrentScope) {
+        continue;
+      }
 
-        assert field.getType()
-            == lookup.getFieldAccess(correlName).getField().getType();
+      if (prevNs == null) {
+        prevNs = foundNs;
+      } else {
+        assert prevNs == foundNs : "All correlation variables should resolve"
+            + " to the same namespace."
+            + " Prev ns=" + prevNs
+            + ", new ns=" + foundNs;
+      }
 
-        assert pos != -1;
+      int namespaceOffset = 0;
+      if (childNamespaceIndex > 0) {
+        // If not the first child, need to figure out the width
+        // of output types from all the preceding namespaces
+        assert ancestorScope instanceof ListScope;
+        List<SqlValidatorNamespace> children =
+            ((ListScope) ancestorScope).getChildren();
+
+        for (int i = 0; i < childNamespaceIndex; i++) {
+          SqlValidatorNamespace child = children.get(i);
+          namespaceOffset +=
+              child.getRowType().getFieldCount();
+        }
+      }
 
-        if (bb.mapRootRelToFieldProjection.containsKey(bb.root)) {
-          // bb.root is an aggregate and only projects group by
-          // keys.
-          Map<Integer, Integer> exprProjection =
-              bb.mapRootRelToFieldProjection.get(bb.root);
+      RelDataTypeField field =
+          catalogReader.field(foundNs.getRowType(), originalFieldName);
+      int pos = namespaceOffset + field.getIndex();
 
-          // subquery can reference group by keys projected from
-          // the root of the outer relation.
-          if (exprProjection.containsKey(pos)) {
-            pos = exprProjection.get(pos);
-          } else {
-            // correl not grouped
-            throw Util.newInternal(
-                "Identifier '" + originalRelName + "."
-                + originalFieldName + "' is not a group expr");
-          }
-        }
+      assert field.getType()
+          == lookup.getFieldAccess(correlName).getField().getType();
 
-        requiredColumns.set(pos);
-        correlNames.add(correlName);
-      }
+      assert pos != -1;
 
-      if (!correlNames.isEmpty()) {
-        if (correlNames.size() > 1) {
-          // The same table was referenced more than once.
-          // So we deduplicate
-          RelShuttle dedup =
-              new DeduplicateCorrelateVariables(rexBuilder,
-                  correlNames.get(0),
-                  ImmutableSet.copyOf(Util.skip(correlNames)));
-          rightRel = rightRel.accept(dedup);
-        }
-        LogicalCorrelate corr = LogicalCorrelate.create(leftRel, rightRel,
-            new CorrelationId(correlNames.get(0)), requiredColumns.build(),
-            SemiJoinType.of(joinType));
-        if (!joinCond.isAlwaysTrue()) {
-          return RelOptUtil.createFilter(corr, joinCond);
+      if (bb.mapRootRelToFieldProjection.containsKey(bb.root)) {
+        // bb.root is an aggregate and only projects group by
+        // keys.
+        Map<Integer, Integer> exprProjection =
+            bb.mapRootRelToFieldProjection.get(bb.root);
+
+        // subquery can reference group by keys projected from
+        // the root of the outer relation.
+        if (exprProjection.containsKey(pos)) {
+          pos = exprProjection.get(pos);
+        } else {
+          // correl not grouped
+          throw new AssertionError("Identifier '" + originalRelName + "."
+              + originalFieldName + "' is not a group expr");
         }
-        return corr;
       }
-    }
 
-    final Join originalJoin =
-        (Join) RelFactories.DEFAULT_JOIN_FACTORY.createJoin(leftRel, rightRel,
-            joinCond, joinType, ImmutableSet.<String>of(), false);
+      requiredColumns.set(pos);
+      correlNames.add(correlName);
+    }
 
-    return RelOptUtil.pushDownJoinConditions(originalJoin);
-  }
+    if (correlNames.isEmpty()) {
+      // None of the correlating variables originated in this scope.
+      return null;
+    }
 
-  private static boolean containsGet(RexNode node) {
-    try {
-      node.accept(
-          new RexVisitorImpl<Void>(true) {
-            @Override public Void visitCall(RexCall call) {
-              if (call.getOperator() == RexBuilder.GET_OPERATOR) {
-                throw Util.FoundOne.NULL;
-              }
-              return super.visitCall(call);
-            }
-          });
-      return false;
-    } catch (Util.FoundOne e) {
-      return true;
+    RelNode r = r0;
+    if (correlNames.size() > 1) {
+      // The same table was referenced more than once.
+      // So we deduplicate
+      r = DeduplicateCorrelateVariables.go(rexBuilder, correlNames.get(0),
+          Util.skip(correlNames), r0);
     }
+    return new CorrelationUse(correlNames.get(0), requiredColumns.build(), r);
   }
 
   /**
@@ -2293,8 +2303,8 @@ public class SqlToRelConverter {
    * @return true if the subquery is non-correlated.
    */
   private boolean isSubQueryNonCorrelated(RelNode subq, Blackboard bb) {
-    Set<String> correlatedVariables = RelOptUtil.getVariablesUsed(subq);
-    for (String correlName : correlatedVariables) {
+    Set<CorrelationId> correlatedVariables = RelOptUtil.getVariablesUsed(subq);
+    for (CorrelationId correlName : correlatedVariables) {
       DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
       String originalRelName = lookup.getOriginalRelName();
 
@@ -2837,7 +2847,14 @@ public class SqlToRelConverter {
    * @return Whether to trim unused fields
    */
   public boolean isTrimUnusedFields() {
-    return trimUnusedFields;
+    // To work around [CALCITE-842] "Decorrelator gets field offsets confused if
+    // fields have been trimmed", if expansion is disabled, trim fields after
+    // expansion and decorrelation.
+    return trimUnusedFields && expand;
+  }
+
+  public void setExpand(boolean expand) {
+    this.expand = expand;
   }
 
   /**
@@ -3262,9 +3279,9 @@ public class SqlToRelConverter {
       qualified = SqlQualified.create(null, 1, null, identifier);
     }
     RexNode e = bb.lookupExp(qualified);
-    final String correlationName;
+    final CorrelationId correlationName;
     if (e instanceof RexCorrelVariable) {
-      correlationName = ((RexCorrelVariable) e).getName();
+      correlationName = ((RexCorrelVariable) e).id;
     } else {
       correlationName = null;
     }
@@ -3279,10 +3296,9 @@ public class SqlToRelConverter {
     }
 
     if (null != correlationName) {
-      // REVIEW: make mapCorrelateVariableToRexNode map to RexFieldAccess
       assert e instanceof RexFieldAccess;
       final RexNode prev =
-          bb.mapCorrelateVariableToRexNode.put(correlationName, e);
+          bb.mapCorrelateToRex.put(correlationName, (RexFieldAccess) e);
       assert prev == null;
     }
     return e;
@@ -3444,8 +3460,8 @@ public class SqlToRelConverter {
               ret,
               relNode,
               rexBuilder.makeLiteral(true),
+              ImmutableSet.<CorrelationId>of(),
               JoinRelType.INNER,
-              ImmutableSet.<String>of(),
               false);
     }
     return ret;
@@ -3634,16 +3650,6 @@ public class SqlToRelConverter {
     // ?
   }
 
-  private String createCorrel() {
-    int n = nextCorrel++;
-    return CORREL_PREFIX + n;
-  }
-
-  private int getCorrelOrdinal(String correlName) {
-    assert correlName.startsWith(CORREL_PREFIX);
-    return Integer.parseInt(correlName.substring(CORREL_PREFIX.length()));
-  }
-
   //~ Inner Classes ----------------------------------------------------------
 
   /**
@@ -3658,7 +3664,7 @@ public class SqlToRelConverter {
     private final Map<String, RexNode> nameToNodeMap;
     public RelNode root;
     private List<RelNode> inputs;
-    private final Map<String, RexNode> mapCorrelateVariableToRexNode =
+    private final Map<CorrelationId, RexFieldAccess> mapCorrelateToRex =
         new HashMap<>();
 
     final List<RelNode> cursors = new ArrayList<>();
@@ -3928,7 +3934,7 @@ public class SqlToRelConverter {
         assert isParent;
         DeferredLookup lookup =
             new DeferredLookup(this, qualified.identifier.names.get(0));
-        String correlName = createCorrel();
+        final CorrelationId correlName = cluster.createCorrel();
         mapCorrelToDeferred.put(correlName, lookup);
         final RelDataType rowType = foundNs.getRowType();
         return rexBuilder.makeCorrel(rowType, correlName);
@@ -4045,6 +4051,59 @@ public class SqlToRelConverter {
       // expressions.
       final SqlKind kind = expr.getKind();
       final SubQuery subQuery;
+      if (!expand) {
+        final SqlCall call;
+        final SqlNode query;
+        final RelRoot root;
+        switch (kind) {
+        case IN:
+          call = (SqlCall) expr;
+          query = call.operand(1);
+          if (!(query instanceof SqlNodeList)) {
+            final SqlInOperator op = (SqlInOperator) call.getOperator();
+            root = convertQueryRecursive(query, false, null);
+            final SqlNode operand = call.operand(0);
+            List<SqlNode> nodes;
+            switch (operand.getKind()) {
+            case ROW:
+              nodes = ((SqlCall) operand).getOperandList();
+              break;
+            default:
+              nodes = ImmutableList.of(operand);
+            }
+            final ImmutableList.Builder<RexNode> builder =
+                ImmutableList.builder();
+            for (SqlNode node : nodes) {
+              builder.add(convertExpression(node));
+            }
+            final RexSubQuery in = RexSubQuery.in(root.rel, builder.build());
+            return op.isNotIn()
+                ? rexBuilder.makeCall(SqlStdOperatorTable.NOT, in)
+                : in;
+          }
+          break;
+
+        case EXISTS:
+          call = (SqlCall) expr;
+          query = Iterables.getOnlyElement(call.getOperandList());
+          root = convertQueryRecursive(query, false, null);
+          RelNode rel = root.rel;
+          while (rel instanceof Project
+              || rel instanceof Sort
+              && ((Sort) rel).fetch == null
+              && ((Sort) rel).offset == null) {
+            rel = ((SingleRel) rel).getInput();
+          }
+          return RexSubQuery.exists(rel);
+
+        case SCALAR_QUERY:
+          call = (SqlCall) expr;
+          query = Iterables.getOnlyElement(call.getOperandList());
+          root = convertQueryRecursive(query, false, null);
+          return RexSubQuery.scalar(root.rel);
+        }
+      }
+
       switch (kind) {
       case CURSOR:
       case IN:
@@ -4252,8 +4311,8 @@ public class SqlToRelConverter {
       this.originalRelName = originalRelName;
     }
 
-    public RexFieldAccess getFieldAccess(String name) {
-      return (RexFieldAccess) bb.mapCorrelateVariableToRexNode.get(name);
+    public RexFieldAccess getFieldAccess(CorrelationId name) {
+      return (RexFieldAccess) bb.mapCorrelateToRex.get(name);
     }
 
     public String getOriginalRelName() {
@@ -4934,6 +4993,21 @@ public class SqlToRelConverter {
       return call.getOperator().acceptCall(this, call);
     }
   }
+
+  /** Use of a row as a correlating variable by a given relational
+   * expression. */
+  private static class CorrelationUse {
+    private final CorrelationId id;
+    private final ImmutableBitSet requiredColumns;
+    private final RelNode r;
+
+    CorrelationUse(CorrelationId id, ImmutableBitSet requiredColumns,
+        RelNode r) {
+      this.id = id;
+      this.requiredColumns = requiredColumns;
+      this.r = r;
+    }
+  }
 }
 
 // End SqlToRelConverter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/tools/Programs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Programs.java b/core/src/main/java/org/apache/calcite/tools/Programs.java
index fd249a4..472563e 100644
--- a/core/src/main/java/org/apache/calcite/tools/Programs.java
+++ b/core/src/main/java/org/apache/calcite/tools/Programs.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.tools;
 
 import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.interpreter.NoneToBindableConverterRule;
 import org.apache.calcite.plan.RelOptCostImpl;
 import org.apache.calcite.plan.RelOptPlanner;
@@ -30,6 +31,7 @@ import org.apache.calcite.plan.hep.HepProgramBuilder;
 import org.apache.calcite.prepare.CalcitePrepareImpl;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -54,7 +56,11 @@ import org.apache.calcite.rel.rules.ProjectMergeRule;
 import org.apache.calcite.rel.rules.ProjectToCalcRule;
 import org.apache.calcite.rel.rules.SemiJoinRule;
 import org.apache.calcite.rel.rules.SortProjectTransposeRule;
+import org.apache.calcite.rel.rules.SubQueryRemoveRule;
 import org.apache.calcite.rel.rules.TableScanRule;
+import org.apache.calcite.sql2rel.RelDecorrelator;
+import org.apache.calcite.sql2rel.RelFieldTrimmer;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
@@ -98,6 +104,13 @@ public class Programs {
   public static final Program CALC_PROGRAM =
       hep(CALC_RULES, true, new DefaultRelMetadataProvider());
 
+  /** Program that expands sub-queries. */
+  public static final Program SUB_QUERY_PROGRAM =
+      hep(
+          ImmutableList.of((RelOptRule) SubQueryRemoveRule.FILTER,
+              SubQueryRemoveRule.PROJECT,
+              SubQueryRemoveRule.JOIN), true, new DefaultRelMetadataProvider());
+
   public static final ImmutableSet<RelOptRule> RULE_SET =
       ImmutableSet.of(
           EnumerableRules.ENUMERABLE_JOIN_RULE,
@@ -258,6 +271,7 @@ public class Programs {
 
   /** Returns the standard program used by Prepare. */
   public static Program standard() {
+
     final Program program1 =
         new Program() {
           public RelNode run(RelOptPlanner planner, RelNode rel,
@@ -276,11 +290,14 @@ public class Programs {
           }
         };
 
-    // Second planner pass to do physical "tweaks". This the first time that
-    // EnumerableCalcRel is introduced.
-    final Program program2 = CALC_PROGRAM;
+    return sequence(SUB_QUERY_PROGRAM,
+        new DecorrelateProgram(),
+        new TrimFieldsProgram(),
+        program1,
 
-    return sequence(program1, program2);
+        // Second planner pass to do physical "tweaks". This the first time that
+        // EnumerableCalcRel is introduced.
+        CALC_PROGRAM);
   }
 
   /** Program backed by a {@link RuleSet}. */
@@ -323,6 +340,35 @@ public class Programs {
       return rel;
     }
   }
+
+  /** Program that de-correlates a query.
+   *
+   * <p>To work around
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-842">[CALCITE-842]
+   * Decorrelator gets field offsets confused if fields have been trimmed</a>,
+   * disable field-trimming in {@link SqlToRelConverter}, and run
+   * {@link TrimFieldsProgram} after this program. */
+  private static class DecorrelateProgram implements Program {
+    public RelNode run(RelOptPlanner planner, RelNode rel,
+        RelTraitSet requiredOutputTraits) {
+      final CalciteConnectionConfig config =
+          planner.getContext().unwrap(CalciteConnectionConfig.class);
+      if (config != null && config.forceDecorrelate()) {
+        return RelDecorrelator.decorrelateQuery(rel);
+      }
+      return rel;
+    }
+  }
+
+  /** Program that trims fields. */
+  private static class TrimFieldsProgram implements Program {
+    public RelNode run(RelOptPlanner planner, RelNode rel,
+        RelTraitSet requiredOutputTraits) {
+      final RelBuilder relBuilder =
+          RelFactories.LOGICAL_BUILDER.create(rel.getCluster(), null);
+      return new RelFieldTrimmer(null, relBuilder).trim(rel);
+    }
+  }
 }
 
 // End Programs.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 eb9ad70..916f0dd 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -29,13 +29,13 @@ import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -47,6 +47,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.server.CalciteServerStatement;
+import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
@@ -54,6 +55,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Stacks;
@@ -66,6 +68,7 @@ import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
 import java.math.BigDecimal;
@@ -75,6 +78,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Objects;
+import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
@@ -111,6 +115,7 @@ public class RelBuilder {
   private final RelFactories.SetOpFactory setOpFactory;
   private final RelFactories.JoinFactory joinFactory;
   private final RelFactories.SemiJoinFactory semiJoinFactory;
+  private final RelFactories.CorrelateFactory correlateFactory;
   private final RelFactories.ValuesFactory valuesFactory;
   private final RelFactories.TableScanFactory scanFactory;
   private final List<Frame> stack = new ArrayList<>();
@@ -143,6 +148,9 @@ public class RelBuilder {
     this.semiJoinFactory =
         Util.first(context.unwrap(RelFactories.SemiJoinFactory.class),
             RelFactories.DEFAULT_SEMI_JOIN_FACTORY);
+    this.correlateFactory =
+        Util.first(context.unwrap(RelFactories.CorrelateFactory.class),
+            RelFactories.DEFAULT_CORRELATE_FACTORY);
     this.valuesFactory =
         Util.first(context.unwrap(RelFactories.ValuesFactory.class),
             RelFactories.DEFAULT_VALUES_FACTORY);
@@ -738,7 +746,7 @@ public class RelBuilder {
       final String name2 = inferAlias(exprList, node);
       names.add(Util.first(name, name2));
     }
-    if (ProjectRemoveRule.isIdentity(exprList, peek().getRowType())) {
+    if (RexUtil.isIdentity(exprList, peek().getRowType())) {
       return this;
     }
     final RelDataType inputRowType = peek().getRowType();
@@ -983,15 +991,44 @@ public class RelBuilder {
    * conditions. */
   public RelBuilder join(JoinRelType joinType,
       Iterable<? extends RexNode> conditions) {
+    return join(joinType,
+        RexUtil.composeConjunction(cluster.getRexBuilder(), conditions, false),
+        ImmutableSet.<CorrelationId>of());
+  }
+
+  public RelBuilder join(JoinRelType joinType, RexNode condition) {
+    return join(joinType, condition, ImmutableSet.<CorrelationId>of());
+  }
+
+  /** Creates a {@link org.apache.calcite.rel.core.Join} with correlating
+   * variables. */
+  public RelBuilder join(JoinRelType joinType, RexNode condition,
+      Set<CorrelationId> variablesSet) {
     final Frame right = Stacks.pop(stack);
     final Frame left = Stacks.pop(stack);
-    final RelNode join = joinFactory.createJoin(left.rel, right.rel,
-        RexUtil.composeConjunction(cluster.getRexBuilder(), conditions, false),
-        joinType, ImmutableSet.<String>of(), false);
+    final RelNode join;
+    final boolean correlate = variablesSet.size() == 1;
+    if (correlate) {
+      final CorrelationId id = Iterables.getOnlyElement(variablesSet);
+      final ImmutableBitSet requiredColumns =
+          RelOptUtil.correlationColumns(id, right.rel);
+      if (!RelOptUtil.notContainsCorrelation(left.rel, id, Litmus.IGNORE)) {
+        throw new IllegalArgumentException("variable " + id
+            + " must not be used by left input to correlation");
+      }
+      join = correlateFactory.createCorrelate(left.rel, right.rel, id,
+          requiredColumns, SemiJoinType.of(joinType));
+    } else {
+      join = joinFactory.createJoin(left.rel, right.rel, condition,
+          variablesSet, joinType, false);
+    }
     final List<Pair<String, RelDataType>> pairs = new ArrayList<>();
     pairs.addAll(left.right);
     pairs.addAll(right.right);
     Stacks.push(stack, new Frame(join, ImmutableList.copyOf(pairs)));
+    if (correlate) {
+      filter(condition);
+    }
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/main/java/org/apache/calcite/util/Bug.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Bug.java b/core/src/main/java/org/apache/calcite/util/Bug.java
index 6248684..884b967 100644
--- a/core/src/main/java/org/apache/calcite/util/Bug.java
+++ b/core/src/main/java/org/apache/calcite/util/Bug.java
@@ -175,6 +175,11 @@ public abstract class Bug {
    * Detect cycles when computing statistics</a> is fixed. */
   public static final boolean CALCITE_794_FIXED = false;
 
+  /** Whether
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1045">[CALCITE-1045]
+   * Decorrelate sub-queries in Project and Join</a> is fixed. */
+  public static final boolean CALCITE_1045_FIXED = false;
+
   /**
    * Use this to flag temporary code.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 4b36b51..e37aa2c 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -142,7 +142,7 @@ public class CalciteAssert {
           return this;
         }
 
-        @Override public AssertThat with(String property, String value) {
+        @Override public AssertThat with(String property, Object value) {
           return this;
         }
 
@@ -808,7 +808,7 @@ public class CalciteAssert {
       return x;
     }
 
-    public AssertThat with(String property, String value) {
+    public AssertThat with(String property, Object value) {
       return new AssertThat(connectionFactory.with(property, value));
     }
 
@@ -977,7 +977,7 @@ public class CalciteAssert {
   public abstract static class ConnectionFactory {
     public abstract Connection createConnection() throws SQLException;
 
-    public ConnectionFactory with(String property, String value) {
+    public ConnectionFactory with(String property, Object value) {
       throw new UnsupportedOperationException();
     }
 
@@ -1107,10 +1107,10 @@ public class CalciteAssert {
       return connection;
     }
 
-    public ConnectionFactory with(String property, String value) {
+    public ConnectionFactory with(String property, Object value) {
       ImmutableMap.Builder<String, String> b = ImmutableMap.builder();
       b.putAll(this.map);
-      b.put(property, value);
+      b.put(property, value.toString());
       return new MapConnectionFactory(b.build(), postProcessors);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index 18a44ac..3615003 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -103,20 +103,20 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.dept d \n"
             + "on e.deptno = d.deptno")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], DEPTNO=[$2], DNAME=[$4])\n"
-            + "    JdbcJoin(condition=[=($2, $3)], joinType=[inner])\n"
-            + "      JdbcProject(EMPNO=[$0], ENAME=[$1], DEPTNO=[$7])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "  JdbcProject(EMPNO=[$2], ENAME=[$3], DEPTNO=[$4], DNAME=[$1])\n"
+            + "    JdbcJoin(condition=[=($4, $0)], joinType=[inner])\n"
             + "      JdbcProject(DEPTNO=[$0], DNAME=[$1])\n"
-            + "        JdbcTableScan(table=[[SCOTT, DEPT]])")
+            + "        JdbcTableScan(table=[[SCOTT, DEPT]])\n"
+            + "      JdbcProject(EMPNO=[$0], ENAME=[$1], DEPTNO=[$7])\n"
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
-        .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
-            + "\"t\".\"DEPTNO\", \"t0\".\"DNAME\"\n"
-            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"DEPTNO\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
-            + "INNER JOIN (SELECT \"DEPTNO\", \"DNAME\"\n"
-            + "FROM \"SCOTT\".\"DEPT\") AS \"t0\" "
+        .planHasSql("SELECT \"t0\".\"EMPNO\", \"t0\".\"ENAME\", "
+            + "\"t0\".\"DEPTNO\", \"t\".\"DNAME\"\n"
+            + "FROM (SELECT \"DEPTNO\", \"DNAME\"\n"
+            + "FROM \"SCOTT\".\"DEPT\") AS \"t\"\n"
+            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"DEPTNO\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" "
             + "ON \"t\".\"DEPTNO\" = \"t0\".\"DEPTNO\"");
   }
 
@@ -129,17 +129,20 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.salgrade s \n"
             + "on e.sal > s.losal and e.sal < s.hisal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], GRADE=[$3])\n"
-            + "    JdbcJoin(condition=[AND(>($2, $4), <($2, $5))], joinType=[inner])\n"
+            + "  JdbcProject(EMPNO=[$3], ENAME=[$4], GRADE=[$0])\n"
+            + "    JdbcJoin(condition=[AND(>($5, $1), <($5, $2))], joinType=[inner])\n"
+            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], SAL=[$5])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
-            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])")
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
         .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
             + "\"SALGRADE\".\"GRADE\"\n"
-            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"SAL\"\nFROM \"SCOTT\".\"EMP\") AS \"t\"\n"
-            + "INNER JOIN \"SCOTT\".\"SALGRADE\" ON \"t\".\"SAL\" > \"SALGRADE\".\"LOSAL\" AND \"t\".\"SAL\" < \"SALGRADE\".\"HISAL\"");
+            + "FROM \"SCOTT\".\"SALGRADE\"\n"
+            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"SAL\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t\" "
+            + "ON \"SALGRADE\".\"LOSAL\" < \"t\".\"SAL\" "
+            + "AND \"SALGRADE\".\"HISAL\" > \"t\".\"SAL\"");
   }
 
   @Test public void testNonEquiJoinReverseConditionPlan() {
@@ -148,18 +151,20 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.salgrade s \n"
             + "on s.losal <= e.sal and s.hisal >= e.sal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], GRADE=[$3])\n"
-            + "    JdbcJoin(condition=[AND(<=($4, $2), >=($5, $2))], joinType=[inner])\n"
+            + "  JdbcProject(EMPNO=[$3], ENAME=[$4], GRADE=[$0])\n"
+            + "    JdbcJoin(condition=[AND(<=($1, $5), >=($2, $5))], joinType=[inner])\n"
+            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], SAL=[$5])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
-            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])")
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
         .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
             + "\"SALGRADE\".\"GRADE\"\n"
-            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"SAL\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
-            + "INNER JOIN \"SCOTT\".\"SALGRADE\" ON \"t\".\"SAL\" >= \"SALGRADE\".\"LOSAL\" AND \"t\".\"SAL\" <= \"SALGRADE\".\"HISAL\"");
+            + "FROM \"SCOTT\".\"SALGRADE\"\n"
+            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"SAL\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t\" "
+            + "ON \"SALGRADE\".\"LOSAL\" <= \"t\".\"SAL\" "
+            + "AND \"SALGRADE\".\"HISAL\" >= \"t\".\"SAL\"");
   }
 
   @Test public void testMixedJoinPlan() {
@@ -168,20 +173,19 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.emp m on  \n"
             + "e.mgr = m.empno and e.sal > m.sal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], EMPNO0=[$0], ENAME0=[$1])\n"
-            + "    JdbcJoin(condition=[AND(=($2, $4), >($3, $5))], joinType=[inner])\n"
-            + "      JdbcProject(EMPNO=[$0], ENAME=[$1], MGR=[$3], SAL=[$5])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "  JdbcProject(EMPNO=[$2], ENAME=[$3], EMPNO0=[$2], ENAME0=[$3])\n"
+            + "    JdbcJoin(condition=[AND(=($4, $0), >($5, $1))], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], SAL=[$5])\n"
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "      JdbcProject(EMPNO=[$0], ENAME=[$1], MGR=[$3], SAL=[$5])\n"
             + "        JdbcTableScan(table=[[SCOTT, EMP]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
-        .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
-            + "\"t\".\"EMPNO\" AS \"EMPNO0\", \"t\".\"ENAME\" AS \"ENAME0\"\n"
-            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"MGR\", \"SAL\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
-            + "INNER JOIN (SELECT \"EMPNO\", \"SAL\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" ON \"t\".\"MGR\" = \"t0\".\"EMPNO\" AND \"t\".\"SAL\" > \"t0\".\"SAL\"");
+        .planHasSql("SELECT \"t0\".\"EMPNO\", \"t0\".\"ENAME\", "
+            + "\"t0\".\"EMPNO\" AS \"EMPNO0\", \"t0\".\"ENAME\" AS \"ENAME0\"\n"
+            + "FROM (SELECT \"EMPNO\", \"SAL\"\nFROM \"SCOTT\".\"EMP\") AS \"t\"\n"
+            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"MGR\", \"SAL\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" ON \"t\".\"EMPNO\" = \"t0\".\"MGR\" AND \"t\".\"SAL\" < \"t0\".\"SAL\"");
   }
 
   @Test public void testMixedJoinWithOrPlan() {
@@ -190,20 +194,23 @@ public class JdbcAdapterTest {
             + "from scott.emp e inner join scott.emp m on  \n"
             + "e.mgr = m.empno and (e.sal > m.sal or m.hiredate > e.hiredate)")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$0], ENAME=[$1], EMPNO0=[$0], ENAME0=[$1])\n"
-            + "    JdbcJoin(condition=[AND(=($2, $5), OR(>($4, $7), >($6, $3)))], joinType=[inner])\n"
-            + "      JdbcProject(EMPNO=[$0], ENAME=[$1], MGR=[$3], HIREDATE=[$4], SAL=[$5])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "  JdbcProject(EMPNO=[$3], ENAME=[$4], EMPNO0=[$3], ENAME0=[$4])\n"
+            + "    JdbcJoin(condition=[AND(=($5, $0), OR(>($7, $2), >($1, $6)))], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], HIREDATE=[$4], SAL=[$5])\n"
+            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "      JdbcProject(EMPNO=[$0], ENAME=[$1], MGR=[$3], HIREDATE=[$4], SAL=[$5])\n"
             + "        JdbcTableScan(table=[[SCOTT, EMP]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
-        .planHasSql("SELECT \"t\".\"EMPNO\", \"t\".\"ENAME\", "
-            + "\"t\".\"EMPNO\" AS \"EMPNO0\", \"t\".\"ENAME\" AS \"ENAME0\"\n"
-            + "FROM (SELECT \"EMPNO\", \"ENAME\", \"MGR\", \"HIREDATE\", \"SAL\"\n"
+        .planHasSql("SELECT \"t0\".\"EMPNO\", \"t0\".\"ENAME\", "
+            + "\"t0\".\"EMPNO\" AS \"EMPNO0\", \"t0\".\"ENAME\" AS \"ENAME0\"\n"
+            + "FROM (SELECT \"EMPNO\", \"HIREDATE\", \"SAL\"\n"
             + "FROM \"SCOTT\".\"EMP\") AS \"t\"\n"
-            + "INNER JOIN (SELECT \"EMPNO\", \"HIREDATE\", \"SAL\"\n"
-            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" ON \"t\".\"MGR\" = \"t0\".\"EMPNO\" AND (\"t\".\"SAL\" > \"t0\".\"SAL\" OR \"t\".\"HIREDATE\" < \"t0\".\"HIREDATE\")");
+            + "INNER JOIN (SELECT \"EMPNO\", \"ENAME\", \"MGR\", \"HIREDATE\", \"SAL\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t0\" "
+            + "ON \"t\".\"EMPNO\" = \"t0\".\"MGR\" "
+            + "AND (\"t\".\"SAL\" < \"t0\".\"SAL\" "
+            + "OR \"t\".\"HIREDATE\" > \"t0\".\"HIREDATE\")");
   }
 
   @Test public void testJoin3TablesPlan() {
@@ -214,19 +221,26 @@ public class JdbcAdapterTest {
             + "inner join scott.salgrade s \n"
             + "on e.sal > s.losal and e.sal < s.hisal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
-            + "  JdbcProject(EMPNO=[$3], ENAME=[$4], DNAME=[$12], GRADE=[$0])\n"
-            + "    JdbcJoin(condition=[AND(>($8, $1), <($8, $2))], joinType=[inner])\n"
-            + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])\n"
-            + "      JdbcJoin(condition=[=($7, $8)], joinType=[inner])\n"
-            + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
-            + "        JdbcTableScan(table=[[SCOTT, DEPT]])")
+            + "  JdbcProject(EMPNO=[$2], ENAME=[$3], DNAME=[$1], GRADE=[$6])\n"
+            + "    JdbcJoin(condition=[=($5, $0)], joinType=[inner])\n"
+            + "      JdbcProject(DEPTNO=[$0], DNAME=[$1])\n"
+            + "        JdbcTableScan(table=[[SCOTT, DEPT]])\n"
+            + "      JdbcJoin(condition=[AND(>($2, $5), <($2, $6))], joinType=[inner])\n"
+            + "        JdbcProject(EMPNO=[$0], ENAME=[$1], SAL=[$5], DEPTNO=[$7])\n"
+            + "          JdbcTableScan(table=[[SCOTT, EMP]])\n"
+            + "        JdbcTableScan(table=[[SCOTT, SALGRADE]])")
         .runs()
         .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.HSQLDB)
-        .planHasSql("SELECT \"EMP\".\"EMPNO\", \"EMP\".\"ENAME\", "
-            + "\"DEPT\".\"DNAME\", \"SALGRADE\".\"GRADE\"\n"
-            + "FROM \"SCOTT\".\"SALGRADE\"\n"
-            + "INNER JOIN (\"SCOTT\".\"EMP\" INNER JOIN \"SCOTT\".\"DEPT\" ON \"EMP\".\"DEPTNO\" = \"DEPT\".\"DEPTNO\") "
-            + "ON \"SALGRADE\".\"LOSAL\" < \"EMP\".\"SAL\" AND \"SALGRADE\".\"HISAL\" > \"EMP\".\"SAL\"");
+        .planHasSql("SELECT \"t0\".\"EMPNO\", \"t0\".\"ENAME\", "
+            + "\"t\".\"DNAME\", \"SALGRADE\".\"GRADE\"\n"
+            + "FROM (SELECT \"DEPTNO\", \"DNAME\"\n"
+            + "FROM \"SCOTT\".\"DEPT\") AS \"t\"\n"
+            + "INNER JOIN ((SELECT \"EMPNO\", \"ENAME\", \"SAL\", \"DEPTNO\"\n"
+            + "FROM \"SCOTT\".\"EMP\") AS \"t0\"\n"
+            + "INNER JOIN \"SCOTT\".\"SALGRADE\" "
+            + "ON \"t0\".\"SAL\" > \"SALGRADE\".\"LOSAL\" "
+            + "AND \"t0\".\"SAL\" < \"SALGRADE\".\"HISAL\") "
+            + "ON \"t\".\"DEPTNO\" = \"t0\".\"DEPTNO\"");
   }
 
   @Test public void testCrossJoinWithJoinKeyPlan() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 243f903..ac9c56b 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -2786,13 +2786,12 @@ public class JdbcTest {
             + "  where \"empid\" < 150)")
         .convertContains(""
             + "LogicalProject(deptno=[$0], name=[$1], employees=[$2], location=[$3])\n"
-            + "  LogicalJoin(condition=[=($0, $4)], joinType=[inner])\n"
-            + "    EnumerableTableScan(table=[[hr, depts]])\n"
-            + "    LogicalAggregate(group=[{0}])\n"
-            + "      LogicalProject(deptno=[$1])\n"
-            + "        LogicalFilter(condition=[<($0, 150)])\n"
-            + "          LogicalProject(empid=[$0], deptno=[$1])\n"
-            + "            EnumerableTableScan(table=[[hr, emps]])")
+            + "  LogicalFilter(condition=[IN($0, {\n"
+            + "LogicalProject(deptno=[$1])\n"
+            + "  LogicalFilter(condition=[<($0, 150)])\n"
+            + "    EnumerableTableScan(table=[[hr, emps]])\n"
+            + "})])\n"
+            + "    EnumerableTableScan(table=[[hr, depts]])")
         .explainContains(""
             + "EnumerableSemiJoin(condition=[=($0, $5)], joinType=[inner])\n"
             + "  EnumerableTableScan(table=[[hr, depts]])\n"
@@ -3331,17 +3330,14 @@ public class JdbcTest {
 
   /** Query that reads no columns from either underlying table. */
   @Test public void testCountStar() {
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true);
-         final TryThreadLocal.Memo memo = Prepare.THREAD_EXPAND.push(true)) {
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       CalciteAssert.hr()
           .query("select count(*) c from \"hr\".\"emps\", \"hr\".\"depts\"")
           .convertContains("LogicalAggregate(group=[{}], C=[COUNT()])\n"
-              + "  LogicalProject(DUMMY=[0])\n"
+              + "  LogicalProject($f0=[0])\n"
               + "    LogicalJoin(condition=[true], joinType=[inner])\n"
-              + "      LogicalProject(DUMMY=[0])\n"
-              + "        EnumerableTableScan(table=[[hr, emps]])\n"
-              + "      LogicalProject(DUMMY=[0])\n"
-              + "        EnumerableTableScan(table=[[hr, depts]])");
+              + "      EnumerableTableScan(table=[[hr, emps]])\n"
+              + "      EnumerableTableScan(table=[[hr, depts]])");
     }
   }
 
@@ -4154,11 +4150,12 @@ public class JdbcTest {
     // Rows are deemed "equal to" the current row per the ORDER BY clause.
     // If there is no ORDER BY clause, CURRENT ROW has the same effect as
     // UNBOUNDED FOLLOWING; that is, no filtering effect at all.
-    checkOuter("select *,\n"
-            + " count(*) over (partition by deptno) as m1,\n"
-            + " count(*) over (partition by deptno order by ename) as m2,\n"
-            + " count(*) over () as m3\n"
-            + "from emp",
+    final String sql = "select *,\n"
+        + " count(*) over (partition by deptno) as m1,\n"
+        + " count(*) over (partition by deptno order by ename) as m2,\n"
+        + " count(*) over () as m3\n"
+        + "from emp";
+    withEmpDept(sql).returnsUnordered(
         "ENAME=Adam ; DEPTNO=50; GENDER=M; M1=2; M2=1; M3=9",
         "ENAME=Alice; DEPTNO=30; GENDER=F; M1=2; M2=1; M3=9",
         "ENAME=Bob  ; DEPTNO=10; GENDER=M; M1=2; M2=1; M3=9",
@@ -4187,9 +4184,7 @@ public class JdbcTest {
   /** Tests that field-trimming creates a project near the table scan, in a
    * query with windowed-aggregation. */
   @Test public void testTrimFieldsOver() throws Exception {
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true);
-         final TryThreadLocal.Memo memo = Prepare.THREAD_EXPAND.push(true)) {
-      Util.discard(memo);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       // The correct plan has a project on a filter on a project on a scan.
       CalciteAssert.hr()
           .query("select \"name\",\n"
@@ -4422,33 +4417,42 @@ public class JdbcTest {
   @Test public void testNotInEmptyQuery() {
     // RHS is empty, therefore returns all rows from emp, including the one
     // with deptno = NULL.
-    checkOuter("select deptno from emp where deptno not in (\n"
-        + "select deptno from dept where deptno = -1)",
-        "DEPTNO=null",
-        "DEPTNO=10",
-        "DEPTNO=10",
-        "DEPTNO=20",
-        "DEPTNO=30",
-        "DEPTNO=30",
-        "DEPTNO=50",
-        "DEPTNO=50",
-        "DEPTNO=60");
+    final String sql = "select deptno from emp where deptno not in (\n"
+        + "select deptno from dept where deptno = -1)";
+    withEmpDept(sql)
+//        .explainContains("EnumerableCalc(expr#0..2=[{inputs}], "
+//            + "expr#3=[IS NOT NULL($t2)], expr#4=[true], "
+//            + "expr#5=[IS NULL($t0)], expr#6=[null], expr#7=[false], "
+//            + "expr#8=[CASE($t3, $t4, $t5, $t6, $t7)], expr#9=[NOT($t8)], "
+//            + "EXPR$1=[$t0], $condition=[$t9])")
+        .returnsUnordered("DEPTNO=null",
+            "DEPTNO=10",
+            "DEPTNO=10",
+            "DEPTNO=20",
+            "DEPTNO=30",
+            "DEPTNO=30",
+            "DEPTNO=50",
+            "DEPTNO=50",
+            "DEPTNO=60");
   }
 
   @Test public void testNotInQuery() {
     // None of the rows from RHS is NULL.
-    checkOuter("select deptno from emp where deptno not in (\n"
-        + "select deptno from dept)",
-        "DEPTNO=50",
-        "DEPTNO=50",
-        "DEPTNO=60");
+    final String sql = "select deptno from emp where deptno not in (\n"
+        + "select deptno from dept)";
+    withEmpDept(sql)
+        .returnsUnordered("DEPTNO=50",
+            "DEPTNO=50",
+            "DEPTNO=60");
   }
 
   @Test public void testNotInQueryWithNull() {
     // There is a NULL on the RHS, and '10 not in (20, null)' yields unknown
     // (similarly for every other value of deptno), so no rows are returned.
-    checkOuter("select deptno from emp where deptno not in (\n"
-        + "select deptno from emp)");
+    final String sql = "select deptno from emp where deptno not in (\n"
+        + "select deptno from emp)";
+    withEmpDept(sql)
+        .returnsCount(0);
   }
 
   @Test public void testTrim() {
@@ -4472,10 +4476,17 @@ public class JdbcTest {
   }
 
   @Test public void testExistsCorrelated() {
-    CalciteAssert.hr()
-        .query("select*from \"hr\".\"emps\" where exists (\n"
-            + " select 1 from \"hr\".\"depts\"\n"
-            + " where \"emps\".\"deptno\"=\"depts\".\"deptno\")")
+    final String sql = "select*from \"hr\".\"emps\" where exists (\n"
+        + " select 1 from \"hr\".\"depts\"\n"
+        + " where \"emps\".\"deptno\"=\"depts\".\"deptno\")";
+    final String plan = ""
+        + "LogicalProject(empid=[$0], deptno=[$1], name=[$2], salary=[$3], commission=[$4])\n"
+        + "  LogicalFilter(condition=[EXISTS({\n"
+        + "LogicalFilter(condition=[=($cor0.deptno, $0)])\n"
+        + "  EnumerableTableScan(table=[[hr, depts]])\n"
+        + "})], variablesSet=[[$cor0]])\n"
+        + "    EnumerableTableScan(table=[[hr, emps]])\n";
+    CalciteAssert.hr().query(sql).convertContains(plan)
         .returnsUnordered(
             "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000",
             "empid=150; deptno=10; name=Sebastian; salary=7000.0; commission=null",
@@ -4483,14 +4494,51 @@ public class JdbcTest {
   }
 
   @Test public void testNotExistsCorrelated() {
+    final String plan = "PLAN="
+        + "EnumerableCalc(expr#0..5=[{inputs}], expr#6=[IS NOT NULL($t5)], expr#7=[true], expr#8=[false], expr#9=[CASE($t6, $t7, $t8)], expr#10=[NOT($t9)], proj#0..4=[{exprs}], $condition=[$t10])\n"
+        + "  EnumerableCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{1}])\n"
+        + "    EnumerableTableScan(table=[[hr, emps]])\n"
+        + "    EnumerableAggregate(group=[{0}])\n"
+        + "      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], expr#5=[$cor0], expr#6=[$t5.deptno], expr#7=[=($t6, $t0)], i=[$t4], $condition=[$t7])\n"
+        + "        EnumerableTableScan(table=[[hr, depts]])\n";
+    final String sql = "select * from \"hr\".\"emps\" where not exists (\n"
+        + " select 1 from \"hr\".\"depts\"\n"
+        + " where \"emps\".\"deptno\"=\"depts\".\"deptno\")";
     CalciteAssert.hr()
-        .query("select * from \"hr\".\"emps\" where not exists (\n"
-            + " select 1 from \"hr\".\"depts\"\n"
-            + " where \"emps\".\"deptno\"=\"depts\".\"deptno\")")
+        .with("forceDecorrelate", false)
+        .query(sql)
+        .explainContains(plan)
         .returnsUnordered(
             "empid=200; deptno=20; name=Eric; salary=8000.0; commission=500");
   }
 
+  /** Manual expansion of EXISTS in {@link #testNotExistsCorrelated()}. */
+  @Test public void testNotExistsCorrelated2() {
+    final String sql = "select * from \"hr\".\"emps\" as e left join lateral (\n"
+        + " select distinct true as i\n"
+        + " from \"hr\".\"depts\"\n"
+        + " where e.\"deptno\"=\"depts\".\"deptno\") on true";
+    final String explain = ""
+        + "EnumerableCalc(expr#0..6=[{inputs}], proj#0..4=[{exprs}], I=[$t6])\n"
+        + "  EnumerableJoin(condition=[=($1, $5)], joinType=[left])\n"
+        + "    EnumerableTableScan(table=[[hr, emps]])\n"
+        + "    EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])\n"
+        + "      EnumerableAggregate(group=[{0}])\n"
+        + "        EnumerableJoin(condition=[=($0, $1)], joinType=[inner])\n"
+        + "          EnumerableAggregate(group=[{1}])\n"
+        + "            EnumerableTableScan(table=[[hr, emps]])\n"
+        + "          EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])\n"
+        + "            EnumerableTableScan(table=[[hr, depts]])";
+    CalciteAssert.hr()
+        .query(sql)
+        .explainContains(explain)
+        .returnsUnordered(
+            "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000; I=true",
+            "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250; I=true",
+            "empid=150; deptno=10; name=Sebastian; salary=7000.0; commission=null; I=true",
+            "empid=200; deptno=20; name=Eric; salary=8000.0; commission=500; I=null");
+  }
+
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-313">[CALCITE-313]
    * Query decorrelation fails</a>. */
@@ -4518,15 +4566,17 @@ public class JdbcTest {
    * <p>Note that there should be an extra row "empid=200; deptno=20;
    * DNAME=null" but left join doesn't work.</p> */
   @Test public void testScalarSubQuery() {
-    CalciteAssert.hr()
-        .query("select \"empid\", \"deptno\",\n"
-            + " (select \"name\" from \"hr\".\"depts\"\n"
-            + "  where \"deptno\" = e.\"deptno\") as dname\n"
-            + "from \"hr\".\"emps\" as e")
-        .returnsUnordered("empid=100; deptno=10; DNAME=Sales",
-            "empid=110; deptno=10; DNAME=Sales",
-            "empid=150; deptno=10; DNAME=Sales",
-            "empid=200; deptno=20; DNAME=null");
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      CalciteAssert.hr()
+          .query("select \"empid\", \"deptno\",\n"
+              + " (select \"name\" from \"hr\".\"depts\"\n"
+              + "  where \"deptno\" = e.\"deptno\") as dname\n"
+              + "from \"hr\".\"emps\" as e")
+          .returnsUnordered("empid=100; deptno=10; DNAME=Sales",
+              "empid=110; deptno=10; DNAME=Sales",
+              "empid=150; deptno=10; DNAME=Sales",
+              "empid=200; deptno=20; DNAME=null");
+    }
   }
 
   @Ignore("CALCITE-559 Correlated subquery will hit exception in Calcite")
@@ -4587,8 +4637,9 @@ public class JdbcTest {
    * join conditions in various flavors of outer join. Results are verified
    * against MySQL (except full join, which MySQL does not support). */
   @Test public void testVariousOuter() {
-    checkOuter(
-        "select * from emp join dept on emp.deptno = dept.deptno",
+    final String sql =
+        "select * from emp join dept on emp.deptno = dept.deptno";
+    withEmpDept(sql).returnsUnordered(
         "ENAME=Alice; DEPTNO=30; GENDER=F; DEPTNO0=30; DNAME=Engineering",
         "ENAME=Bob  ; DEPTNO=10; GENDER=M; DEPTNO0=10; DNAME=Sales      ",
         "ENAME=Eric ; DEPTNO=20; GENDER=M; DEPTNO0=20; DNAME=Marketing  ",
@@ -4596,7 +4647,7 @@ public class JdbcTest {
         "ENAME=Susan; DEPTNO=30; GENDER=F; DEPTNO0=30; DNAME=Engineering");
   }
 
-  private void checkOuter(String sql, String... lines) {
+  private CalciteAssert.AssertQuery withEmpDept(String sql) {
     // Append a 'WITH' clause that supplies EMP and DEPT tables like this:
     //
     // drop table emp;
@@ -4616,7 +4667,7 @@ public class JdbcTest {
     // insert into dept values (20, 'Marketing');
     // insert into dept values (30, 'Engineering');
     // insert into dept values (40, 'Empty');
-    CalciteAssert.that()
+    return CalciteAssert.that()
         .query("with\n"
             + "  emp(ename, deptno, gender) as (values\n"
             + "    ('Jane', 10, 'F'),\n"
@@ -4633,15 +4684,14 @@ public class JdbcTest {
             + "    (20, 'Marketing'),\n"
             + "    (30, 'Engineering'),\n"
             + "    (40, 'Empty'))\n"
-            + sql)
-        .returnsUnordered(lines);
+            + sql);
   }
 
   /** Runs the dummy script, which is checked in empty but which you may
    * use as scratch space during development. */
   // Do not add '@Ignore'; just remember not to commit changes to dummy.iq
   @Test public void testRunDummy() throws Exception {
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(false)) {
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
       checkRun("sql/dummy.iq");
     }
   }
@@ -4669,7 +4719,9 @@ public class JdbcTest {
       // Oracle as the JDBC data source.
       return;
     }
-    checkRun("sql/misc.iq");
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      checkRun("sql/misc.iq");
+    }
   }
 
   @Test public void testRunSequence() throws Exception {
@@ -4681,7 +4733,9 @@ public class JdbcTest {
   }
 
   @Test public void testRunScalar() throws Exception {
-    checkRun("sql/scalar.iq");
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      checkRun("sql/scalar.iq");
+    }
   }
 
   @Test public void testRunSubquery() throws Exception {
@@ -4724,6 +4778,8 @@ public class JdbcTest {
                   switch (v) {
                   case "calcite794":
                     return Bug.CALCITE_794_FIXED;
+                  case "calcite1045":
+                    return Bug.CALCITE_1045_FIXED;
                   }
                   return null;
                 }
@@ -4830,19 +4886,21 @@ public class JdbcTest {
   }
 
   @Test public void testScalarSubQueryInCase() {
-    CalciteAssert.hr()
-        .query("select e.\"name\",\n"
-            + " (CASE e.\"deptno\"\n"
-            + "  WHEN (Select \"deptno\" from \"hr\".\"depts\" d\n"
-            + "        where d.\"deptno\" = e.\"deptno\")\n"
-            + "  THEN (Select d.\"name\" from \"hr\".\"depts\" d\n"
-            + "        where d.\"deptno\" = e.\"deptno\")\n"
-            + "  ELSE 'DepartmentNotFound'  END) AS DEPTNAME\n"
-            + "from \"hr\".\"emps\" e")
-        .returnsUnordered("name=Bill; DEPTNAME=Sales",
-            "name=Eric; DEPTNAME=DepartmentNotFound",
-            "name=Sebastian; DEPTNAME=Sales",
-            "name=Theodore; DEPTNAME=Sales");
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      CalciteAssert.hr()
+          .query("select e.\"name\",\n"
+              + " (CASE e.\"deptno\"\n"
+              + "  WHEN (Select \"deptno\" from \"hr\".\"depts\" d\n"
+              + "        where d.\"deptno\" = e.\"deptno\")\n"
+              + "  THEN (Select d.\"name\" from \"hr\".\"depts\" d\n"
+              + "        where d.\"deptno\" = e.\"deptno\")\n"
+              + "  ELSE 'DepartmentNotFound'  END) AS DEPTNAME\n"
+              + "from \"hr\".\"emps\" e")
+          .returnsUnordered("name=Bill; DEPTNAME=Sales",
+              "name=Eric; DEPTNAME=DepartmentNotFound",
+              "name=Sebastian; DEPTNAME=Sales",
+              "name=Theodore; DEPTNAME=Sales");
+    }
   }
 
   @Test public void testScalarSubQueryInCase2() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/java/org/apache/calcite/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
index 397ad3a..30ff1e1 100644
--- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
@@ -231,10 +231,10 @@ public class LatticeTest {
       foodmartModel()
           .query("select count(*) from \"adhoc\".\"star\"")
           .convertMatches(
-              CalciteAssert.checkRel(
-                  "LogicalAggregate(group=[{}], EXPR$0=[COUNT()])\n"
-                      + "  LogicalProject(DUMMY=[0])\n"
-                      + "    StarTableScan(table=[[adhoc, star]])\n",
+              CalciteAssert.checkRel(""
+                  + "LogicalAggregate(group=[{}], EXPR$0=[COUNT()])\n"
+                  + "  LogicalProject($f0=[0])\n"
+                  + "    StarTableScan(table=[[adhoc, star]])\n",
                   counter));
     } catch (RuntimeException e) {
       assertThat(Throwables.getStackTraceAsString(e),
@@ -429,24 +429,29 @@ public class LatticeTest {
 
   /** Tests a query that is created within {@link #testTileAlgorithm()}. */
   @Test public void testJG() {
+    final String sql = ""
+        + "SELECT \"s\".\"unit_sales\", \"p\".\"recyclable_package\", \"t\".\"the_day\", \"t\".\"the_year\", \"t\".\"quarter\", \"pc\".\"product_family\", COUNT(*) AS \"m0\", SUM(\"s\".\"store_sales\") AS \"m1\", SUM(\"s\".\"unit_sales\") AS \"m2\"\n"
+        + "FROM \"foodmart\".\"sales_fact_1997\" AS \"s\"\n"
+        + "JOIN \"foodmart\".\"product\" AS \"p\" ON \"s\".\"product_id\" = \"p\".\"product_id\"\n"
+        + "JOIN \"foodmart\".\"time_by_day\" AS \"t\" ON \"s\".\"time_id\" = \"t\".\"time_id\"\n"
+        + "JOIN \"foodmart\".\"product_class\" AS \"pc\" ON \"p\".\"product_class_id\" = \"pc\".\"product_class_id\"\n"
+        + "GROUP BY \"s\".\"unit_sales\", \"p\".\"recyclable_package\", \"t\".\"the_day\", \"t\".\"the_year\", \"t\".\"quarter\", \"pc\".\"product_family\"";
+    final String explain = "JdbcToEnumerableConverter\n"
+        + "  JdbcAggregate(group=[{3, 6, 8, 9, 10, 12}], m0=[COUNT()], m1=[$SUM0($2)], m2=[$SUM0($3)])\n"
+        + "    JdbcJoin(condition=[=($4, $11)], joinType=[inner])\n"
+        + "      JdbcJoin(condition=[=($1, $7)], joinType=[inner])\n"
+        + "        JdbcJoin(condition=[=($0, $5)], joinType=[inner])\n"
+        + "          JdbcProject(product_id=[$0], time_id=[$1], store_sales=[$5], unit_sales=[$7])\n"
+        + "            JdbcTableScan(table=[[foodmart, sales_fact_1997]])\n"
+        + "          JdbcProject(product_class_id=[$0], product_id=[$1], recyclable_package=[$8])\n"
+        + "            JdbcTableScan(table=[[foodmart, product]])\n"
+        + "        JdbcProject(time_id=[$0], the_day=[$2], the_year=[$4], quarter=[$8])\n"
+        + "          JdbcTableScan(table=[[foodmart, time_by_day]])\n"
+        + "      JdbcProject(product_class_id=[$0], product_family=[$4])\n"
+        + "        JdbcTableScan(table=[[foodmart, product_class]])";
     CalciteAssert.that().with(CalciteAssert.Config.JDBC_FOODMART)
-        .query(
-            "SELECT \"s\".\"unit_sales\", \"p\".\"recyclable_package\", \"t\".\"the_day\", \"t\".\"the_year\", \"t\".\"quarter\", \"pc\".\"product_family\", COUNT(*) AS \"m0\", SUM(\"s\".\"store_sales\") AS \"m1\", SUM(\"s\".\"unit_sales\") AS \"m2\"\n"
-                + "FROM \"foodmart\".\"sales_fact_1997\" AS \"s\"\n"
-                + "JOIN \"foodmart\".\"product\" AS \"p\" ON \"s\".\"product_id\" = \"p\".\"product_id\"\n"
-                + "JOIN \"foodmart\".\"time_by_day\" AS \"t\" ON \"s\".\"time_id\" = \"t\".\"time_id\"\n"
-                + "JOIN \"foodmart\".\"product_class\" AS \"pc\" ON \"p\".\"product_class_id\" = \"pc\".\"product_class_id\"\n"
-                + "GROUP BY \"s\".\"unit_sales\", \"p\".\"recyclable_package\", \"t\".\"the_day\", \"t\".\"the_year\", \"t\".\"quarter\", \"pc\".\"product_family\"")
-        .explainContains(
-            "JdbcToEnumerableConverter\n"
-                + "  JdbcAggregate(group=[{7, 16, 25, 27, 31, 37}], m0=[COUNT()], m1=[$SUM0($5)], m2=[$SUM0($7)])\n"
-                + "    JdbcJoin(condition=[=($8, $33)], joinType=[inner])\n"
-                + "      JdbcJoin(condition=[=($1, $23)], joinType=[inner])\n"
-                + "        JdbcJoin(condition=[=($0, $9)], joinType=[inner])\n"
-                + "          JdbcTableScan(table=[[foodmart, sales_fact_1997]])\n"
-                + "          JdbcTableScan(table=[[foodmart, product]])\n"
-                + "        JdbcTableScan(table=[[foodmart, time_by_day]])\n"
-                + "      JdbcTableScan(table=[[foodmart, product_class]])");
+        .query(sql)
+        .explainContains(explain);
   }
 
   /** Tests a query that uses no columns from the fact table. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
index 0ebd616..1e03f87 100644
--- a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
@@ -25,7 +25,6 @@ import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.Predicate1;
-import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
 import org.apache.calcite.linq4j.tree.Primitive;
@@ -148,9 +147,8 @@ public class ReflectiveSchemaTest {
                     Types.of(Enumerable.class, Employee.class),
                     null,
                     LINQ4J_AS_ENUMERABLE_METHOD,
-                    Arrays.<Expression>asList(
-                        Expressions.constant(
-                            new JdbcTest.HrSchema().emps))), "asQueryable"),
+                    Expressions.constant(new JdbcTest.HrSchema().emps)),
+                "asQueryable"),
             Employee.class)
             .select(
                 Expressions.<Function1<Employee, Integer>>lambda(
@@ -447,9 +445,24 @@ public class ReflectiveSchemaTest {
         + "where \"wrapperBoolean\"")
         .returns("C=0\n");
     with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
+        + "where \"wrapperBoolean\" is true")
+        .returns("C=0\n");
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
         + "where \"wrapperBoolean\" is not true")
         .returns("C=2\n");
     with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
+        + "where \"wrapperBoolean\" is false")
+        .returns("C=1\n");
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
+        + "where \"wrapperBoolean\" is not false")
+        .returns("C=1\n");
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
+        + "where \"wrapperBoolean\" is null")
+        .returns("C=1\n");
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
+        + "where \"wrapperBoolean\" is not null")
+        .returns("C=1\n");
+    with.query("select count(*) as c from \"s\".\"everyTypes\"\n"
         + "where \"primitiveInt\" > 0")
         .returns("C=1\n");
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 7df8dcf..bb53728 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -16,11 +16,13 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Exchange;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.TableFunctionScan;
@@ -44,6 +46,7 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 
 import org.junit.Test;
 
@@ -51,6 +54,7 @@ import java.sql.PreparedStatement;
 import java.util.Arrays;
 import java.util.List;
 
+import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
@@ -770,6 +774,24 @@ public class RelBuilderTest {
     assertThat(str(root), is(expected));
   }
 
+  @Test public void testCorrelationFails() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+    builder.scan("EMP");
+    final RelOptCluster cluster = builder.peek().getCluster();
+    final CorrelationId id = cluster.createCorrel();
+    final RexNode v =
+        builder.getRexBuilder().makeCorrel(builder.peek().getRowType(), id);
+    try {
+      builder.filter(builder.equals(builder.field(0), v))
+          .scan("DEPT")
+          .join(JoinRelType.INNER, builder.literal(true), ImmutableSet.of(id));
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(),
+          containsString("variable $cor0 must not be used by left input to correlation"));
+    }
+  }
+
   @Test public void testAlias() {
     // Equivalent SQL:
     //   SELECT *

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 0e76a40..f8c4b2f 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
@@ -1160,7 +1161,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     // Join
     final LogicalJoin join =
         LogicalJoin.create(empScan, deptProject, rexBuilder.makeLiteral(true),
-            JoinRelType.INNER, ImmutableSet.<String>of());
+            ImmutableSet.<CorrelationId>of(), JoinRelType.INNER);
     rowSize = RelMetadataQuery.getAverageRowSize(join);
     columnSizes = RelMetadataQuery.getAverageColumnSizes(join);
     assertThat(columnSizes.size(), equalTo(13));

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 92bae3e..6300579 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -78,6 +78,7 @@ import org.apache.calcite.rel.rules.SemiJoinRule;
 import org.apache.calcite.rel.rules.SortJoinTransposeRule;
 import org.apache.calcite.rel.rules.SortProjectTransposeRule;
 import org.apache.calcite.rel.rules.SortUnionTransposeRule;
+import org.apache.calcite.rel.rules.SubQueryRemoveRule;
 import org.apache.calcite.rel.rules.TableScanRule;
 import org.apache.calcite.rel.rules.UnionToDistinctRule;
 import org.apache.calcite.rel.rules.ValuesReduceRule;
@@ -110,7 +111,7 @@ import static org.junit.Assert.assertTrue;
  * translated into relational algebra and then fed into a
  * {@link org.apache.calcite.plan.hep.HepPlanner}. The planner fires the rule on
  * every
- * pattern match in a depth-first left-to-right preorder traversal of the tree
+ * pattern match in a depth-first left-to-right pre-order traversal of the tree
  * for as long as the rule continues to succeed in applying its transform. (For
  * rules which call transformTo more than once, only the last result is used.)
  * The plan before and after "optimization" is diffed against a .ref file using
@@ -1995,7 +1996,7 @@ public class RelOptRulesTest extends RelOptTestBase {
    * Wrong collation trait in SortJoinTransposeRule for right joins</a>. */
   @Test public void testSortJoinTranspose4() {
     // Create a customized test with RelCollation trait in the test cluster.
-    Tester tester = new TesterImpl(getDiffRepos(), true, false, null) {
+    Tester tester = new TesterImpl(getDiffRepos(), true, true, false, null) {
       @Override public RelOptPlanner createPlanner() {
         return new MockRelOptPlanner() {
           @Override public List<RelTraitDef> getRelTraitDefs() {
@@ -2061,6 +2062,160 @@ public class RelOptRulesTest extends RelOptTestBase {
         + "having count(*) > 3";
     checkPlanning(new HepPlanner(program), sql);
   }
+
+  private Sql checkSubQuery(String sql) {
+    final HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(SubQueryRemoveRule.PROJECT)
+        .addRuleInstance(SubQueryRemoveRule.FILTER)
+        .addRuleInstance(SubQueryRemoveRule.JOIN)
+        .build();
+    return sql(sql).with(new HepPlanner(program)).expand(false);
+  }
+
+  /** Tests expanding a sub-query, specifically an uncorrelated scalar
+   * sub-query in a project (SELECT clause). */
+  @Test public void testExpandProjectScalar() throws Exception {
+    final String sql = "select empno,\n"
+        + "  (select deptno from sales.emp where empno < 20) as d\n"
+        + "from sales.emp";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandProjectIn() throws Exception {
+    final String sql = "select empno,\n"
+        + "  deptno in (select deptno from sales.emp where empno < 20) as d\n"
+        + "from sales.emp";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandProjectInNullable() throws Exception {
+    final String sql = "with e2 as (\n"
+        + "  select empno, case when true then deptno else null end as deptno\n"
+        + "  from sales.emp)\n"
+        + "select empno,\n"
+        + "  deptno in (select deptno from e2 where empno < 20) as d\n"
+        + "from e2";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandProjectInComposite() throws Exception {
+    final String sql = "select empno, (empno, deptno) in (\n"
+        + "    select empno, deptno from sales.emp where empno < 20) as d\n"
+        + "from sales.emp";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandProjectExists() throws Exception {
+    final String sql = "select empno,\n"
+        + "  exists (select deptno from sales.emp where empno < 20) as d\n"
+        + "from sales.emp";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandFilterScalar() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp\n"
+        + "where (select deptno from sales.emp where empno < 20)\n"
+        + " < (select deptno from sales.emp where empno > 100)\n"
+        + "or emp.sal < 100";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandFilterIn() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp\n"
+        + "where deptno in (select deptno from sales.emp where empno < 20)\n"
+        + "or emp.sal < 100";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandFilterInComposite() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp\n"
+        + "where (empno, deptno) in (\n"
+        + "  select empno, deptno from sales.emp where empno < 20)\n"
+        + "or emp.sal < 100";
+    checkSubQuery(sql).check();
+  }
+
+  /** An IN filter that requires full 3-value logic (true, false, unknown). */
+  @Test public void testExpandFilterIn3Value() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp\n"
+        + "where empno\n"
+        + " < case deptno in (select case when true then deptno else null end\n"
+        + "                   from sales.emp where empno < 20)\n"
+        + "   when true then 10\n"
+        + "   when false then 20\n"
+        + "   else 30\n"
+        + "   end";
+    checkSubQuery(sql).check();
+  }
+
+  /** An EXISTS filter that can be converted into true/false. */
+  @Test public void testExpandFilterExists() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp\n"
+        + "where exists (select deptno from sales.emp where empno < 20)\n"
+        + "or emp.sal < 100";
+    checkSubQuery(sql).check();
+  }
+
+  /** An EXISTS filter that can be converted into a semi-join. */
+  @Test public void testExpandFilterExistsSimple() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp\n"
+        + "where exists (select deptno from sales.emp where empno < 20)";
+    checkSubQuery(sql).check();
+  }
+
+  /** An EXISTS filter that can be converted into a semi-join. */
+  @Test public void testExpandFilterExistsSimpleAnd() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp\n"
+        + "where exists (select deptno from sales.emp where empno < 20)\n"
+        + "and emp.sal < 100";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandJoinScalar() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp left join sales.dept\n"
+        + "on (select deptno from sales.emp where empno < 20)\n"
+        + " < (select deptno from sales.emp where empno > 100)";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandJoinIn() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp left join sales.dept\n"
+        + "on emp.deptno in (select deptno from sales.emp where empno < 20)";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandJoinInComposite() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp left join sales.dept\n"
+        + "on (emp.empno, dept.deptno) in (\n"
+        + "  select empno, deptno from sales.emp where empno < 20)";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testExpandJoinExists() throws Exception {
+    final String sql = "select empno\n"
+        + "from sales.emp left join sales.dept\n"
+        + "on exists (select deptno from sales.emp where empno < 20)";
+    checkSubQuery(sql).check();
+  }
+
+  @Test public void testWhereInCorrelated() {
+    final String sql = "select empno from emp as e\n"
+        + "join dept as d using (deptno)\n"
+        + "where e.sal in (\n"
+        + "  select e2.sal from emp as e2 where e2.deptno > e.deptno)";
+    checkSubQuery(sql).check();
+  }
+
 }
 
 // End RelOptRulesTest.java


[28/50] [abbrv] calcite git commit: Fix NullPointerException in SqlJoin.toString

Posted by jh...@apache.org.
Fix NullPointerException in SqlJoin.toString


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

Branch: refs/heads/branch-release
Commit: cd92b77c476f7b095b330507402bfc6c3208e300
Parents: 239babd
Author: Julian Hyde <jh...@apache.org>
Authored: Sat Jan 9 12:18:29 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:25 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/sql/SqlJoin.java    | 14 +++---
 .../java/org/apache/calcite/sql/SqlWriter.java  |  5 ++
 .../calcite/sql/pretty/SqlPrettyWriter.java     | 50 ++++++++------------
 3 files changed, 34 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/cd92b77c/core/src/main/java/org/apache/calcite/sql/SqlJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJoin.java b/core/src/main/java/org/apache/calcite/sql/SqlJoin.java
index fc909c4..12b1fb0 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJoin.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJoin.java
@@ -21,6 +21,8 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
+
 import java.util.List;
 
 /**
@@ -58,18 +60,15 @@ public class SqlJoin extends SqlCall {
       SqlNode condition) {
     super(pos);
     this.left = left;
-    this.natural = natural;
-    this.joinType = joinType;
+    this.natural = Preconditions.checkNotNull(natural);
+    this.joinType = Preconditions.checkNotNull(joinType);
     this.right = right;
-    this.conditionType = conditionType;
+    this.conditionType = Preconditions.checkNotNull(conditionType);
     this.condition = condition;
 
     assert natural.getTypeName() == SqlTypeName.BOOLEAN;
-    assert conditionType != null;
     assert conditionType.symbolValue() instanceof JoinConditionType;
-    assert joinType != null;
     assert joinType.symbolValue() instanceof JoinType;
-
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -196,6 +195,8 @@ public class SqlJoin extends SqlCall {
         int rightPrec) {
       final SqlJoin join = (SqlJoin) call;
 
+      final SqlWriter.Frame joinFrame =
+          writer.startList(SqlWriter.FrameTypeEnum.JOIN);
       join.left.unparse(
           writer,
           leftPrec,
@@ -249,6 +250,7 @@ public class SqlJoin extends SqlCall {
           throw Util.unexpected(join.getConditionType());
         }
       }
+      writer.endList(joinFrame);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cd92b77c/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWriter.java b/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
index 8e72026..d45c831 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
@@ -194,6 +194,11 @@ public interface SqlWriter {
     FROM_LIST,
 
     /**
+     * Pair-wise join.
+     */
+    JOIN(false),
+
+    /**
      * WHERE clause.
      */
     WHERE_LIST,

http://git-wip-us.apache.org/repos/asf/calcite/blob/cd92b77c/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
index 2be197e..0d78656 100644
--- a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
@@ -25,6 +25,8 @@ import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteLogger;
 
+import com.google.common.base.Throwables;
+
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.lang.reflect.InvocationTargetException;
@@ -34,6 +36,7 @@ import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
 import java.util.logging.Logger;
@@ -306,11 +309,10 @@ public class SqlPrettyWriter implements SqlWriter {
     final Bean properties = getBean();
     final String[] propertyNames = properties.getPropertyNames();
     int count = 0;
-    for (int i = 0; i < propertyNames.length; i++) {
-      String key = propertyNames[i];
+    for (String key : propertyNames) {
       final Object value = bean.get(key);
       final Object defaultValue = DEFAULT_BEAN.get(key);
-      if (com.google.common.base.Objects.equal(value, defaultValue)) {
+      if (Objects.equals(value, defaultValue)) {
         continue;
       }
       if (count++ > 0) {
@@ -327,8 +329,7 @@ public class SqlPrettyWriter implements SqlWriter {
     resetSettings();
     final Bean bean = getBean();
     final String[] propertyNames = bean.getPropertyNames();
-    for (int i = 0; i < propertyNames.length; i++) {
-      String propertyName = propertyNames[i];
+    for (String propertyName : propertyNames) {
       final String value = properties.getProperty(propertyName);
       if (value != null) {
         bean.set(propertyName, value);
@@ -662,6 +663,7 @@ public class SqlPrettyWriter implements SqlWriter {
             false);
 
       case FROM_LIST:
+      case JOIN:
         return new FrameImpl(
             frameType,
             keyword,
@@ -849,7 +851,7 @@ public class SqlPrettyWriter implements SqlWriter {
 
   protected void whiteSpace() {
     if (needWhitespace) {
-      if (nextWhitespace == NL) {
+      if (nextWhitespace.equals(NL)) {
         newlineAndIndent();
       } else {
         pw.print(nextWhitespace);
@@ -1044,15 +1046,15 @@ public class SqlPrettyWriter implements SqlWriter {
     /**
      * Whether to print a newline before each separator.
      */
-    public boolean newlineBeforeSep;
+    public final boolean newlineBeforeSep;
 
     /**
      * Whether to print a newline after each separator.
      */
-    public boolean newlineAfterSep;
+    public final boolean newlineAfterSep;
     private final boolean newlineBeforeClose;
     private final boolean newlineAfterClose;
-    private boolean newlineAfterOpen;
+    private final boolean newlineAfterOpen;
 
     FrameImpl(
         FrameType frameType,
@@ -1107,18 +1109,14 @@ public class SqlPrettyWriter implements SqlWriter {
    */
   private static class Bean {
     private final SqlPrettyWriter o;
-    private final Map<String, Method> getterMethods =
-        new HashMap<String, Method>();
-    private final Map<String, Method> setterMethods =
-        new HashMap<String, Method>();
+    private final Map<String, Method> getterMethods = new HashMap<>();
+    private final Map<String, Method> setterMethods = new HashMap<>();
 
     Bean(SqlPrettyWriter o) {
       this.o = o;
 
       // Figure out the getter/setter methods for each attribute.
-      final Method[] methods = o.getClass().getMethods();
-      for (int i = 0; i < methods.length; i++) {
-        Method method = methods[i];
+      for (Method method : o.getClass().getMethods()) {
         if (method.getName().startsWith("set")
             && (method.getReturnType() == Void.class)
             && (method.getParameterTypes().length == 1)) {
@@ -1157,13 +1155,9 @@ public class SqlPrettyWriter implements SqlWriter {
     public void set(String name, String value) {
       final Method method = setterMethods.get(name);
       try {
-        method.invoke(
-            o,
-            value);
-      } catch (IllegalAccessException e) {
-        throw Util.newInternal(e);
-      } catch (InvocationTargetException e) {
-        throw Util.newInternal(e);
+        method.invoke(o, value);
+      } catch (IllegalAccessException | InvocationTargetException e) {
+        throw Throwables.propagate(e);
       }
     }
 
@@ -1171,18 +1165,16 @@ public class SqlPrettyWriter implements SqlWriter {
       final Method method = getterMethods.get(name);
       try {
         return method.invoke(o);
-      } catch (IllegalAccessException e) {
-        throw Util.newInternal(e);
-      } catch (InvocationTargetException e) {
-        throw Util.newInternal(e);
+      } catch (IllegalAccessException | InvocationTargetException e) {
+        throw Throwables.propagate(e);
       }
     }
 
     public String[] getPropertyNames() {
-      final Set<String> names = new HashSet<String>();
+      final Set<String> names = new HashSet<>();
       names.addAll(getterMethods.keySet());
       names.addAll(setterMethods.keySet());
-      return (String[]) names.toArray(new String[names.size()]);
+      return names.toArray(new String[names.size()]);
     }
   }
 }


[47/50] [abbrv] calcite git commit: [CALCITE-1057] Add RelMetadataProvider parameter to standard planner Programs

Posted by jh...@apache.org.
[CALCITE-1057] Add RelMetadataProvider parameter to standard planner Programs


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

Branch: refs/heads/branch-release
Commit: af77ec861989c50229c55be1889efcb0031cfd32
Parents: cecef9d
Author: maryannxue <we...@intel.com>
Authored: Thu Jan 14 14:17:18 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Jan 14 14:17:18 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/tools/Programs.java | 27 +++++++++++++++-----
 1 file changed, 20 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/af77ec86/core/src/main/java/org/apache/calcite/tools/Programs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Programs.java b/core/src/main/java/org/apache/calcite/tools/Programs.java
index 472563e..a52387d 100644
--- a/core/src/main/java/org/apache/calcite/tools/Programs.java
+++ b/core/src/main/java/org/apache/calcite/tools/Programs.java
@@ -102,14 +102,11 @@ public class Programs {
 
   /** Program that converts filters and projects to {@link Calc}s. */
   public static final Program CALC_PROGRAM =
-      hep(CALC_RULES, true, new DefaultRelMetadataProvider());
+      calc(new DefaultRelMetadataProvider());
 
   /** Program that expands sub-queries. */
   public static final Program SUB_QUERY_PROGRAM =
-      hep(
-          ImmutableList.of((RelOptRule) SubQueryRemoveRule.FILTER,
-              SubQueryRemoveRule.PROJECT,
-              SubQueryRemoveRule.JOIN), true, new DefaultRelMetadataProvider());
+      subquery(new DefaultRelMetadataProvider());
 
   public static final ImmutableSet<RelOptRule> RULE_SET =
       ImmutableSet.of(
@@ -260,6 +257,17 @@ public class Programs {
     };
   }
 
+  public static Program calc(RelMetadataProvider metadataProvider) {
+    return hep(CALC_RULES, true, metadataProvider);
+  }
+
+  public static Program subquery(RelMetadataProvider metadataProvider) {
+    return hep(
+        ImmutableList.of((RelOptRule) SubQueryRemoveRule.FILTER,
+            SubQueryRemoveRule.PROJECT,
+            SubQueryRemoveRule.JOIN), true, metadataProvider);
+  }
+
   public static Program getProgram() {
     return new Program() {
       public RelNode run(RelOptPlanner planner, RelNode rel,
@@ -271,6 +279,11 @@ public class Programs {
 
   /** Returns the standard program used by Prepare. */
   public static Program standard() {
+    return standard(new DefaultRelMetadataProvider());
+  }
+
+  /** Returns the standard program with user metadata provider. */
+  public static Program standard(RelMetadataProvider metadataProvider) {
 
     final Program program1 =
         new Program() {
@@ -290,14 +303,14 @@ public class Programs {
           }
         };
 
-    return sequence(SUB_QUERY_PROGRAM,
+    return sequence(subquery(metadataProvider),
         new DecorrelateProgram(),
         new TrimFieldsProgram(),
         program1,
 
         // Second planner pass to do physical "tweaks". This the first time that
         // EnumerableCalcRel is introduced.
-        CALC_PROGRAM);
+        calc(metadataProvider));
   }
 
   /** Program backed by a {@link RuleSet}. */


[40/50] [abbrv] calcite git commit: [CALCITE-1040] Differentiate better between arrays and scalars in protobuf

Posted by jh...@apache.org.
[CALCITE-1040] Differentiate better between arrays and scalars in protobuf

The original implementation of ColumnValue tried to always use a repeated
field to serialize the value of a column in a row. This was flawed for
multiple reasons. Instead, go the "struct" approach like TypedValue.

Since we're using protobuf, make the attempt to support the Calcite
1.5 structure (avoid removing the old field).


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

Branch: refs/heads/branch-release
Commit: 361096ba1d0436f588e4c7a3290560059882223f
Parents: 0045e01
Author: Josh Elser <el...@apache.org>
Authored: Mon Jan 11 23:08:48 2016 -0500
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 12 13:47:17 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/avatica/Meta.java   | 102 ++-
 .../apache/calcite/avatica/proto/Common.java    | 849 +++++++++++++++++--
 avatica/src/main/protobuf/common.proto          |   5 +-
 .../org/apache/calcite/avatica/FrameTest.java   |  66 ++
 .../avatica/remote/ProtobufHandlerTest.java     |  13 +-
 .../remote/ProtobufTranslationImplTest.java     |  23 +-
 site/_docs/avatica_protobuf_reference.md        |  15 +-
 7 files changed, 995 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/361096ba/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
index a859608..e9b28d5 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -892,14 +892,17 @@ public interface Meta {
             final Common.ColumnValue.Builder columnBuilder = Common.ColumnValue.newBuilder();
 
             if (element instanceof List) {
+              columnBuilder.setHasArrayValue(true);
               List<?> list = (List<?>) element;
               // Add each element in the list/array to the column's value
               for (Object listItem : list) {
-                columnBuilder.addValue(serializeScalar(listItem));
+                columnBuilder.addArrayValue(serializeScalar(listItem));
               }
             } else {
+              // The default value, but still explicit.
+              columnBuilder.setHasArrayValue(false);
               // Only one value for this column, a scalar.
-              columnBuilder.addValue(serializeScalar(element));
+              columnBuilder.setScalarValue(serializeScalar(element));
             }
 
             // Add value to row
@@ -931,8 +934,7 @@ public interface Meta {
       } else if (element instanceof Long) {
         valueBuilder.setType(Common.Rep.LONG).setNumberValue((Long) element);
       } else if (element instanceof Double) {
-        valueBuilder.setType(Common.Rep.DOUBLE)
-          .setDoubleValue(((Double) element).doubleValue());
+        valueBuilder.setType(Common.Rep.DOUBLE).setDoubleValue((Double) element);
       } else if (element instanceof Float) {
         valueBuilder.setType(Common.Rep.FLOAT).setNumberValue(((Float) element).longValue());
       } else if (element instanceof BigDecimal) {
@@ -967,17 +969,13 @@ public interface Meta {
       for (Common.Row protoRow : proto.getRowsList()) {
         ArrayList<Object> row = new ArrayList<>(protoRow.getValueCount());
         for (Common.ColumnValue protoColumn : protoRow.getValueList()) {
-          Object value;
-          if (protoColumn.getValueCount() > 1) {
-            // Array
-            List<Object> array = new ArrayList<>(protoColumn.getValueCount());
-            for (Common.TypedValue columnValue : protoColumn.getValueList()) {
-              array.add(getScalarValue(columnValue));
-            }
-            value = array;
+          final Object value;
+          if (!isNewStyleColumn(protoColumn)) {
+            // Backward compatibility
+            value = parseOldStyleColumn(protoColumn);
           } else {
-            // Scalar
-            value = getScalarValue(protoColumn.getValue(0));
+            // Current style parsing (separate scalar and array values)
+            value = parseColumn(protoColumn);
           }
 
           row.add(value);
@@ -989,6 +987,82 @@ public interface Meta {
       return new Frame(proto.getOffset(), proto.getDone(), parsedRows);
     }
 
+    /**
+     * Determines whether this message contains the new attributes in the
+     * message. We can't directly test for the negative because our
+     * {@code hasField} trick does not work on repeated fields.
+     *
+     * @param column The protobuf column object
+     * @return True if the message is the new style, false otherwise.
+     */
+    static boolean isNewStyleColumn(Common.ColumnValue column) {
+      final Descriptor desc = column.getDescriptorForType();
+      return ProtobufService.hasField(column, desc, Common.ColumnValue.HAS_ARRAY_VALUE_FIELD_NUMBER)
+          || ProtobufService.hasField(column, desc, Common.ColumnValue.SCALAR_VALUE_FIELD_NUMBER);
+    }
+
+    /**
+     * For Calcite 1.5, we made the mistake of using array length to determine when the value for a
+     * column is a scalar or an array. This method performs the old parsing for backwards
+     * compatibility.
+     *
+     * @param column The protobuf ColumnValue object
+     * @return The parsed value for this column
+     */
+    static Object parseOldStyleColumn(Common.ColumnValue column) {
+      if (column.getValueCount() > 1) {
+        List<Object> array = new ArrayList<>(column.getValueCount());
+        for (Common.TypedValue columnValue : column.getValueList()) {
+          array.add(getScalarValue(columnValue));
+        }
+        return array;
+      } else {
+        return getScalarValue(column.getValue(0));
+      }
+    }
+
+    /**
+     * Parses the value for a ColumnValue using the separated array and scalar attributes.
+     *
+     * @param column The protobuf ColumnValue object
+     * @return The parse value for this column
+     */
+    static Object parseColumn(Common.ColumnValue column) {
+      // Verify that we have one or the other (scalar or array)
+      validateColumnValue(column);
+
+      if (!ProtobufService.hasField(column, column.getDescriptorForType(),
+          Common.ColumnValue.SCALAR_VALUE_FIELD_NUMBER)) {
+        // Array
+        List<Object> array = new ArrayList<>(column.getArrayValueCount());
+        for (Common.TypedValue arrayValue : column.getArrayValueList()) {
+          array.add(getScalarValue(arrayValue));
+        }
+        return array;
+      } else {
+        // Scalar
+        return getScalarValue(column.getScalarValue());
+      }
+    }
+
+    /**
+     * Verifies that a ColumnValue has only a scalar or array value, not both and not neither.
+     *
+     * @param column The protobuf ColumnValue object
+     * @throws IllegalArgumentException When the above condition is not met
+     */
+    static void validateColumnValue(Common.ColumnValue column) {
+      final boolean hasScalar = ProtobufService.hasField(column, column.getDescriptorForType(),
+          Common.ColumnValue.SCALAR_VALUE_FIELD_NUMBER);
+      final boolean hasArrayValue = column.getHasArrayValue();
+
+      // These should always be different
+      if (hasScalar == hasArrayValue) {
+        throw new IllegalArgumentException("A column must have a scalar or array value, not "
+            + (hasScalar ? "both" : "neither"));
+      }
+    }
+
     static Object getScalarValue(Common.TypedValue protoElement) {
       // TODO Should these be primitives or Objects?
       switch (protoElement.getType()) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/361096ba/avatica/src/main/java/org/apache/calcite/avatica/proto/Common.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/proto/Common.java b/avatica/src/main/java/org/apache/calcite/avatica/proto/Common.java
index 1dbe093..7d5cef9 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/proto/Common.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/proto/Common.java
@@ -12091,27 +12091,93 @@ package org.apache.calcite.avatica.proto;
 
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> 
         getValueList();
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     org.apache.calcite.avatica.proto.Common.TypedValue getValue(int index);
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     int getValueCount();
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
         getValueOrBuilderList();
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getValueOrBuilder(
         int index);
+
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> 
+        getArrayValueList();
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.TypedValue getArrayValue(int index);
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    int getArrayValueCount();
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+        getArrayValueOrBuilderList();
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getArrayValueOrBuilder(
+        int index);
+
+    /**
+     * <code>optional bool has_array_value = 3;</code>
+     *
+     * <pre>
+     * Is an array value set?
+     * </pre>
+     */
+    boolean getHasArrayValue();
+
+    /**
+     * <code>optional .TypedValue scalar_value = 4;</code>
+     */
+    boolean hasScalarValue();
+    /**
+     * <code>optional .TypedValue scalar_value = 4;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.TypedValue getScalarValue();
+    /**
+     * <code>optional .TypedValue scalar_value = 4;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getScalarValueOrBuilder();
   }
   /**
    * Protobuf type {@code ColumnValue}
@@ -12130,6 +12196,8 @@ package org.apache.calcite.avatica.proto;
     }
     private ColumnValue() {
       value_ = java.util.Collections.emptyList();
+      arrayValue_ = java.util.Collections.emptyList();
+      hasArrayValue_ = false;
     }
 
     @java.lang.Override
@@ -12164,6 +12232,32 @@ package org.apache.calcite.avatica.proto;
               value_.add(input.readMessage(org.apache.calcite.avatica.proto.Common.TypedValue.parser(), extensionRegistry));
               break;
             }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                arrayValue_ = new java.util.ArrayList<org.apache.calcite.avatica.proto.Common.TypedValue>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              arrayValue_.add(input.readMessage(org.apache.calcite.avatica.proto.Common.TypedValue.parser(), extensionRegistry));
+              break;
+            }
+            case 24: {
+
+              hasArrayValue_ = input.readBool();
+              break;
+            }
+            case 34: {
+              org.apache.calcite.avatica.proto.Common.TypedValue.Builder subBuilder = null;
+              if (scalarValue_ != null) {
+                subBuilder = scalarValue_.toBuilder();
+              }
+              scalarValue_ = input.readMessage(org.apache.calcite.avatica.proto.Common.TypedValue.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(scalarValue_);
+                scalarValue_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -12176,6 +12270,9 @@ package org.apache.calcite.avatica.proto;
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
           value_ = java.util.Collections.unmodifiableList(value_);
         }
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          arrayValue_ = java.util.Collections.unmodifiableList(arrayValue_);
+        }
         makeExtensionsImmutable();
       }
     }
@@ -12191,16 +12288,25 @@ package org.apache.calcite.avatica.proto;
               org.apache.calcite.avatica.proto.Common.ColumnValue.class, org.apache.calcite.avatica.proto.Common.ColumnValue.Builder.class);
     }
 
+    private int bitField0_;
     public static final int VALUE_FIELD_NUMBER = 1;
     private java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> value_;
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> getValueList() {
       return value_;
     }
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     public java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
         getValueOrBuilderList() {
@@ -12208,24 +12314,105 @@ package org.apache.calcite.avatica.proto;
     }
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     public int getValueCount() {
       return value_.size();
     }
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     public org.apache.calcite.avatica.proto.Common.TypedValue getValue(int index) {
       return value_.get(index);
     }
     /**
      * <code>repeated .TypedValue value = 1;</code>
+     *
+     * <pre>
+     * deprecated, use array_value or scalar_value
+     * </pre>
      */
     public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getValueOrBuilder(
         int index) {
       return value_.get(index);
     }
 
+    public static final int ARRAY_VALUE_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> arrayValue_;
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> getArrayValueList() {
+      return arrayValue_;
+    }
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    public java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+        getArrayValueOrBuilderList() {
+      return arrayValue_;
+    }
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    public int getArrayValueCount() {
+      return arrayValue_.size();
+    }
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.TypedValue getArrayValue(int index) {
+      return arrayValue_.get(index);
+    }
+    /**
+     * <code>repeated .TypedValue array_value = 2;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getArrayValueOrBuilder(
+        int index) {
+      return arrayValue_.get(index);
+    }
+
+    public static final int HAS_ARRAY_VALUE_FIELD_NUMBER = 3;
+    private boolean hasArrayValue_;
+    /**
+     * <code>optional bool has_array_value = 3;</code>
+     *
+     * <pre>
+     * Is an array value set?
+     * </pre>
+     */
+    public boolean getHasArrayValue() {
+      return hasArrayValue_;
+    }
+
+    public static final int SCALAR_VALUE_FIELD_NUMBER = 4;
+    private org.apache.calcite.avatica.proto.Common.TypedValue scalarValue_;
+    /**
+     * <code>optional .TypedValue scalar_value = 4;</code>
+     */
+    public boolean hasScalarValue() {
+      return scalarValue_ != null;
+    }
+    /**
+     * <code>optional .TypedValue scalar_value = 4;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.TypedValue getScalarValue() {
+      return scalarValue_ == null ? org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance() : scalarValue_;
+    }
+    /**
+     * <code>optional .TypedValue scalar_value = 4;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getScalarValueOrBuilder() {
+      return getScalarValue();
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -12241,6 +12428,15 @@ package org.apache.calcite.avatica.proto;
       for (int i = 0; i < value_.size(); i++) {
         output.writeMessage(1, value_.get(i));
       }
+      for (int i = 0; i < arrayValue_.size(); i++) {
+        output.writeMessage(2, arrayValue_.get(i));
+      }
+      if (hasArrayValue_ != false) {
+        output.writeBool(3, hasArrayValue_);
+      }
+      if (scalarValue_ != null) {
+        output.writeMessage(4, getScalarValue());
+      }
     }
 
     public int getSerializedSize() {
@@ -12252,6 +12448,18 @@ package org.apache.calcite.avatica.proto;
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, value_.get(i));
       }
+      for (int i = 0; i < arrayValue_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, arrayValue_.get(i));
+      }
+      if (hasArrayValue_ != false) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, hasArrayValue_);
+      }
+      if (scalarValue_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, getScalarValue());
+      }
       memoizedSize = size;
       return size;
     }
@@ -12364,6 +12572,7 @@ package org.apache.calcite.avatica.proto;
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getValueFieldBuilder();
+          getArrayValueFieldBuilder();
         }
       }
       public Builder clear() {
@@ -12374,6 +12583,20 @@ package org.apache.calcite.avatica.proto;
         } else {
           valueBuilder_.clear();
         }
+        if (arrayValueBuilder_ == null) {
+          arrayValue_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          arrayValueBuilder_.clear();
+        }
+        hasArrayValue_ = false;
+
+        if (scalarValueBuilder_ == null) {
+          scalarValue_ = null;
+        } else {
+          scalarValue_ = null;
+          scalarValueBuilder_ = null;
+        }
         return this;
       }
 
@@ -12397,6 +12620,7 @@ package org.apache.calcite.avatica.proto;
       public org.apache.calcite.avatica.proto.Common.ColumnValue buildPartial() {
         org.apache.calcite.avatica.proto.Common.ColumnValue result = new org.apache.calcite.avatica.proto.Common.ColumnValue(this);
         int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
         if (valueBuilder_ == null) {
           if (((bitField0_ & 0x00000001) == 0x00000001)) {
             value_ = java.util.Collections.unmodifiableList(value_);
@@ -12406,6 +12630,22 @@ package org.apache.calcite.avatica.proto;
         } else {
           result.value_ = valueBuilder_.build();
         }
+        if (arrayValueBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            arrayValue_ = java.util.Collections.unmodifiableList(arrayValue_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.arrayValue_ = arrayValue_;
+        } else {
+          result.arrayValue_ = arrayValueBuilder_.build();
+        }
+        result.hasArrayValue_ = hasArrayValue_;
+        if (scalarValueBuilder_ == null) {
+          result.scalarValue_ = scalarValue_;
+        } else {
+          result.scalarValue_ = scalarValueBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -12447,6 +12687,38 @@ package org.apache.calcite.avatica.proto;
             }
           }
         }
+        if (arrayValueBuilder_ == null) {
+          if (!other.arrayValue_.isEmpty()) {
+            if (arrayValue_.isEmpty()) {
+              arrayValue_ = other.arrayValue_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureArrayValueIsMutable();
+              arrayValue_.addAll(other.arrayValue_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.arrayValue_.isEmpty()) {
+            if (arrayValueBuilder_.isEmpty()) {
+              arrayValueBuilder_.dispose();
+              arrayValueBuilder_ = null;
+              arrayValue_ = other.arrayValue_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              arrayValueBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getArrayValueFieldBuilder() : null;
+            } else {
+              arrayValueBuilder_.addAllMessages(other.arrayValue_);
+            }
+          }
+        }
+        if (other.getHasArrayValue() != false) {
+          setHasArrayValue(other.getHasArrayValue());
+        }
+        if (other.hasScalarValue()) {
+          mergeScalarValue(other.getScalarValue());
+        }
         onChanged();
         return this;
       }
@@ -12488,6 +12760,10 @@ package org.apache.calcite.avatica.proto;
 
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> getValueList() {
         if (valueBuilder_ == null) {
@@ -12498,6 +12774,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public int getValueCount() {
         if (valueBuilder_ == null) {
@@ -12508,6 +12788,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public org.apache.calcite.avatica.proto.Common.TypedValue getValue(int index) {
         if (valueBuilder_ == null) {
@@ -12518,6 +12802,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder setValue(
           int index, org.apache.calcite.avatica.proto.Common.TypedValue value) {
@@ -12535,6 +12823,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder setValue(
           int index, org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
@@ -12549,6 +12841,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder addValue(org.apache.calcite.avatica.proto.Common.TypedValue value) {
         if (valueBuilder_ == null) {
@@ -12565,6 +12861,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder addValue(
           int index, org.apache.calcite.avatica.proto.Common.TypedValue value) {
@@ -12582,6 +12882,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder addValue(
           org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
@@ -12596,6 +12900,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder addValue(
           int index, org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
@@ -12610,6 +12918,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder addAllValue(
           java.lang.Iterable<? extends org.apache.calcite.avatica.proto.Common.TypedValue> values) {
@@ -12625,6 +12937,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder clearValue() {
         if (valueBuilder_ == null) {
@@ -12638,6 +12954,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public Builder removeValue(int index) {
         if (valueBuilder_ == null) {
@@ -12651,6 +12971,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public org.apache.calcite.avatica.proto.Common.TypedValue.Builder getValueBuilder(
           int index) {
@@ -12658,6 +12982,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getValueOrBuilder(
           int index) {
@@ -12668,6 +12996,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
            getValueOrBuilderList() {
@@ -12679,6 +13011,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public org.apache.calcite.avatica.proto.Common.TypedValue.Builder addValueBuilder() {
         return getValueFieldBuilder().addBuilder(
@@ -12686,6 +13022,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public org.apache.calcite.avatica.proto.Common.TypedValue.Builder addValueBuilder(
           int index) {
@@ -12694,6 +13034,10 @@ package org.apache.calcite.avatica.proto;
       }
       /**
        * <code>repeated .TypedValue value = 1;</code>
+       *
+       * <pre>
+       * deprecated, use array_value or scalar_value
+       * </pre>
        */
       public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue.Builder> 
            getValueBuilderList() {
@@ -12713,6 +13057,401 @@ package org.apache.calcite.avatica.proto;
         }
         return valueBuilder_;
       }
+
+      private java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> arrayValue_ =
+        java.util.Collections.emptyList();
+      private void ensureArrayValueIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          arrayValue_ = new java.util.ArrayList<org.apache.calcite.avatica.proto.Common.TypedValue>(arrayValue_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> arrayValueBuilder_;
+
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> getArrayValueList() {
+        if (arrayValueBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(arrayValue_);
+        } else {
+          return arrayValueBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public int getArrayValueCount() {
+        if (arrayValueBuilder_ == null) {
+          return arrayValue_.size();
+        } else {
+          return arrayValueBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue getArrayValue(int index) {
+        if (arrayValueBuilder_ == null) {
+          return arrayValue_.get(index);
+        } else {
+          return arrayValueBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder setArrayValue(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (arrayValueBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArrayValueIsMutable();
+          arrayValue_.set(index, value);
+          onChanged();
+        } else {
+          arrayValueBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder setArrayValue(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          arrayValue_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          arrayValueBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder addArrayValue(org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (arrayValueBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArrayValueIsMutable();
+          arrayValue_.add(value);
+          onChanged();
+        } else {
+          arrayValueBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder addArrayValue(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (arrayValueBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArrayValueIsMutable();
+          arrayValue_.add(index, value);
+          onChanged();
+        } else {
+          arrayValueBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder addArrayValue(
+          org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          arrayValue_.add(builderForValue.build());
+          onChanged();
+        } else {
+          arrayValueBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder addArrayValue(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          arrayValue_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          arrayValueBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder addAllArrayValue(
+          java.lang.Iterable<? extends org.apache.calcite.avatica.proto.Common.TypedValue> values) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, arrayValue_);
+          onChanged();
+        } else {
+          arrayValueBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder clearArrayValue() {
+        if (arrayValueBuilder_ == null) {
+          arrayValue_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          arrayValueBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public Builder removeArrayValue(int index) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          arrayValue_.remove(index);
+          onChanged();
+        } else {
+          arrayValueBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder getArrayValueBuilder(
+          int index) {
+        return getArrayValueFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getArrayValueOrBuilder(
+          int index) {
+        if (arrayValueBuilder_ == null) {
+          return arrayValue_.get(index);  } else {
+          return arrayValueBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+           getArrayValueOrBuilderList() {
+        if (arrayValueBuilder_ != null) {
+          return arrayValueBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(arrayValue_);
+        }
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder addArrayValueBuilder() {
+        return getArrayValueFieldBuilder().addBuilder(
+            org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder addArrayValueBuilder(
+          int index) {
+        return getArrayValueFieldBuilder().addBuilder(
+            index, org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .TypedValue array_value = 2;</code>
+       */
+      public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue.Builder> 
+           getArrayValueBuilderList() {
+        return getArrayValueFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+          getArrayValueFieldBuilder() {
+        if (arrayValueBuilder_ == null) {
+          arrayValueBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder>(
+                  arrayValue_,
+                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  getParentForChildren(),
+                  isClean());
+          arrayValue_ = null;
+        }
+        return arrayValueBuilder_;
+      }
+
+      private boolean hasArrayValue_ ;
+      /**
+       * <code>optional bool has_array_value = 3;</code>
+       *
+       * <pre>
+       * Is an array value set?
+       * </pre>
+       */
+      public boolean getHasArrayValue() {
+        return hasArrayValue_;
+      }
+      /**
+       * <code>optional bool has_array_value = 3;</code>
+       *
+       * <pre>
+       * Is an array value set?
+       * </pre>
+       */
+      public Builder setHasArrayValue(boolean value) {
+        
+        hasArrayValue_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool has_array_value = 3;</code>
+       *
+       * <pre>
+       * Is an array value set?
+       * </pre>
+       */
+      public Builder clearHasArrayValue() {
+        
+        hasArrayValue_ = false;
+        onChanged();
+        return this;
+      }
+
+      private org.apache.calcite.avatica.proto.Common.TypedValue scalarValue_ = null;
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> scalarValueBuilder_;
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      public boolean hasScalarValue() {
+        return scalarValueBuilder_ != null || scalarValue_ != null;
+      }
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue getScalarValue() {
+        if (scalarValueBuilder_ == null) {
+          return scalarValue_ == null ? org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance() : scalarValue_;
+        } else {
+          return scalarValueBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      public Builder setScalarValue(org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (scalarValueBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          scalarValue_ = value;
+          onChanged();
+        } else {
+          scalarValueBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      public Builder setScalarValue(
+          org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (scalarValueBuilder_ == null) {
+          scalarValue_ = builderForValue.build();
+          onChanged();
+        } else {
+          scalarValueBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      public Builder mergeScalarValue(org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (scalarValueBuilder_ == null) {
+          if (scalarValue_ != null) {
+            scalarValue_ =
+              org.apache.calcite.avatica.proto.Common.TypedValue.newBuilder(scalarValue_).mergeFrom(value).buildPartial();
+          } else {
+            scalarValue_ = value;
+          }
+          onChanged();
+        } else {
+          scalarValueBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      public Builder clearScalarValue() {
+        if (scalarValueBuilder_ == null) {
+          scalarValue_ = null;
+          onChanged();
+        } else {
+          scalarValue_ = null;
+          scalarValueBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder getScalarValueBuilder() {
+        
+        onChanged();
+        return getScalarValueFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getScalarValueOrBuilder() {
+        if (scalarValueBuilder_ != null) {
+          return scalarValueBuilder_.getMessageOrBuilder();
+        } else {
+          return scalarValue_ == null ?
+              org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance() : scalarValue_;
+        }
+      }
+      /**
+       * <code>optional .TypedValue scalar_value = 4;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+          getScalarValueFieldBuilder() {
+        if (scalarValueBuilder_ == null) {
+          scalarValueBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder>(
+                  getScalarValue(),
+                  getParentForChildren(),
+                  isClean());
+          scalarValue_ = null;
+        }
+        return scalarValueBuilder_;
+      }
       public final Builder setUnknownFields(
           final com.google.protobuf.UnknownFieldSet unknownFields) {
         return this;
@@ -16225,59 +16964,61 @@ package org.apache.calcite.avatica.proto;
       "\003(\0132\014.ColumnValue\"3\n\020DatabaseProperty\022\014\n" +
       "\004name\030\001 \001(\t\022\021\n\tfunctions\030\002 \003(\t\"4\n\013WireMe" +
       "ssage\022\014\n\004name\030\001 \001(\t\022\027\n\017wrapped_message\030\002" +
-      " \001(\014\")\n\013ColumnValue\022\032\n\005value\030\001 \003(\0132\013.Typ",
-      "edValue\"\232\001\n\nTypedValue\022\022\n\004type\030\001 \001(\0162\004.R" +
-      "ep\022\022\n\nbool_value\030\002 \001(\010\022\024\n\014string_value\030\003" +
-      " \001(\t\022\024\n\014number_value\030\004 \001(\022\022\024\n\014bytes_valu" +
-      "es\030\005 \001(\014\022\024\n\014double_value\030\006 \001(\001\022\014\n\004null\030\007" +
-      " \001(\010\"\246\002\n\031MetaDataOperationArgument\022\024\n\014st" +
-      "ring_value\030\001 \001(\t\022\022\n\nbool_value\030\002 \001(\010\022\021\n\t" +
-      "int_value\030\003 \001(\021\022\033\n\023string_array_values\030\004" +
-      " \003(\t\022\030\n\020int_array_values\030\005 \003(\021\0225\n\004type\030\006" +
-      " \001(\0162\'.MetaDataOperationArgument.Argumen" +
-      "tType\"^\n\014ArgumentType\022\n\n\006STRING\020\000\022\010\n\004BOO",
-      "L\020\001\022\007\n\003INT\020\002\022\023\n\017REPEATED_STRING\020\003\022\020\n\014REP" +
-      "EATED_INT\020\004\022\010\n\004NULL\020\005\"\260\001\n\nQueryState\022\030\n\004" +
-      "type\030\001 \001(\0162\n.StateType\022\013\n\003sql\030\002 \001(\t\022\036\n\002o" +
-      "p\030\003 \001(\0162\022.MetaDataOperation\022(\n\004args\030\004 \003(" +
-      "\0132\032.MetaDataOperationArgument\022\020\n\010has_arg" +
-      "s\030\005 \001(\010\022\017\n\007has_sql\030\006 \001(\010\022\016\n\006has_op\030\007 \001(\010" +
-      "*\237\001\n\rStatementType\022\n\n\006SELECT\020\000\022\n\n\006INSERT" +
-      "\020\001\022\n\n\006UPDATE\020\002\022\n\n\006DELETE\020\003\022\n\n\006UPSERT\020\004\022\t" +
-      "\n\005MERGE\020\005\022\r\n\tOTHER_DML\020\006\022\n\n\006CREATE\020\007\022\010\n\004" +
-      "DROP\020\010\022\t\n\005ALTER\020\t\022\r\n\tOTHER_DDL\020\n\022\010\n\004CALL",
-      "\020\013*\342\003\n\003Rep\022\025\n\021PRIMITIVE_BOOLEAN\020\000\022\022\n\016PRI" +
-      "MITIVE_BYTE\020\001\022\022\n\016PRIMITIVE_CHAR\020\002\022\023\n\017PRI" +
-      "MITIVE_SHORT\020\003\022\021\n\rPRIMITIVE_INT\020\004\022\022\n\016PRI" +
-      "MITIVE_LONG\020\005\022\023\n\017PRIMITIVE_FLOAT\020\006\022\024\n\020PR" +
-      "IMITIVE_DOUBLE\020\007\022\013\n\007BOOLEAN\020\010\022\010\n\004BYTE\020\t\022" +
-      "\r\n\tCHARACTER\020\n\022\t\n\005SHORT\020\013\022\013\n\007INTEGER\020\014\022\010" +
-      "\n\004LONG\020\r\022\t\n\005FLOAT\020\016\022\n\n\006DOUBLE\020\017\022\017\n\013BIG_I" +
-      "NTEGER\020\031\022\017\n\013BIG_DECIMAL\020\032\022\021\n\rJAVA_SQL_TI" +
-      "ME\020\020\022\026\n\022JAVA_SQL_TIMESTAMP\020\021\022\021\n\rJAVA_SQL" +
-      "_DATE\020\022\022\022\n\016JAVA_UTIL_DATE\020\023\022\017\n\013BYTE_STRI",
-      "NG\020\024\022\n\n\006STRING\020\025\022\n\n\006NUMBER\020\026\022\n\n\006OBJECT\020\027" +
-      "\022\010\n\004NULL\020\030\022\t\n\005ARRAY\020\033\022\n\n\006STRUCT\020\034\022\014\n\010MUL" +
-      "TISET\020\035*^\n\010Severity\022\024\n\020UNKNOWN_SEVERITY\020" +
-      "\000\022\022\n\016FATAL_SEVERITY\020\001\022\022\n\016ERROR_SEVERITY\020" +
-      "\002\022\024\n\020WARNING_SEVERITY\020\003*\327\004\n\021MetaDataOper" +
-      "ation\022\022\n\016GET_ATTRIBUTES\020\000\022\033\n\027GET_BEST_RO" +
-      "W_IDENTIFIER\020\001\022\020\n\014GET_CATALOGS\020\002\022\036\n\032GET_" +
-      "CLIENT_INFO_PROPERTIES\020\003\022\031\n\025GET_COLUMN_P" +
-      "RIVILEGES\020\004\022\017\n\013GET_COLUMNS\020\005\022\027\n\023GET_CROS" +
-      "S_REFERENCE\020\006\022\025\n\021GET_EXPORTED_KEYS\020\007\022\030\n\024",
-      "GET_FUNCTION_COLUMNS\020\010\022\021\n\rGET_FUNCTIONS\020" +
-      "\t\022\025\n\021GET_IMPORTED_KEYS\020\n\022\022\n\016GET_INDEX_IN" +
-      "FO\020\013\022\024\n\020GET_PRIMARY_KEYS\020\014\022\031\n\025GET_PROCED" +
-      "URE_COLUMNS\020\r\022\022\n\016GET_PROCEDURES\020\016\022\026\n\022GET" +
-      "_PSEUDO_COLUMNS\020\017\022\017\n\013GET_SCHEMAS\020\020\022\031\n\025GE" +
-      "T_SCHEMAS_WITH_ARGS\020\021\022\024\n\020GET_SUPER_TABLE" +
-      "S\020\022\022\023\n\017GET_SUPER_TYPES\020\023\022\030\n\024GET_TABLE_PR" +
-      "IVILEGES\020\024\022\016\n\nGET_TABLES\020\025\022\023\n\017GET_TABLE_" +
-      "TYPES\020\026\022\021\n\rGET_TYPE_INFO\020\027\022\014\n\010GET_UDTS\020\030" +
-      "\022\027\n\023GET_VERSION_COLUMNS\020\031*\"\n\tStateType\022\007",
-      "\n\003SQL\020\000\022\014\n\010METADATA\020\001B\"\n org.apache.calc" +
-      "ite.avatica.protob\006proto3"
+      " \001(\014\"\207\001\n\013ColumnValue\022\032\n\005value\030\001 \003(\0132\013.Ty",
+      "pedValue\022 \n\013array_value\030\002 \003(\0132\013.TypedVal" +
+      "ue\022\027\n\017has_array_value\030\003 \001(\010\022!\n\014scalar_va" +
+      "lue\030\004 \001(\0132\013.TypedValue\"\232\001\n\nTypedValue\022\022\n" +
+      "\004type\030\001 \001(\0162\004.Rep\022\022\n\nbool_value\030\002 \001(\010\022\024\n" +
+      "\014string_value\030\003 \001(\t\022\024\n\014number_value\030\004 \001(" +
+      "\022\022\024\n\014bytes_values\030\005 \001(\014\022\024\n\014double_value\030" +
+      "\006 \001(\001\022\014\n\004null\030\007 \001(\010\"\246\002\n\031MetaDataOperatio" +
+      "nArgument\022\024\n\014string_value\030\001 \001(\t\022\022\n\nbool_" +
+      "value\030\002 \001(\010\022\021\n\tint_value\030\003 \001(\021\022\033\n\023string" +
+      "_array_values\030\004 \003(\t\022\030\n\020int_array_values\030",
+      "\005 \003(\021\0225\n\004type\030\006 \001(\0162\'.MetaDataOperationA" +
+      "rgument.ArgumentType\"^\n\014ArgumentType\022\n\n\006" +
+      "STRING\020\000\022\010\n\004BOOL\020\001\022\007\n\003INT\020\002\022\023\n\017REPEATED_" +
+      "STRING\020\003\022\020\n\014REPEATED_INT\020\004\022\010\n\004NULL\020\005\"\260\001\n" +
+      "\nQueryState\022\030\n\004type\030\001 \001(\0162\n.StateType\022\013\n" +
+      "\003sql\030\002 \001(\t\022\036\n\002op\030\003 \001(\0162\022.MetaDataOperati" +
+      "on\022(\n\004args\030\004 \003(\0132\032.MetaDataOperationArgu" +
+      "ment\022\020\n\010has_args\030\005 \001(\010\022\017\n\007has_sql\030\006 \001(\010\022" +
+      "\016\n\006has_op\030\007 \001(\010*\237\001\n\rStatementType\022\n\n\006SEL" +
+      "ECT\020\000\022\n\n\006INSERT\020\001\022\n\n\006UPDATE\020\002\022\n\n\006DELETE\020",
+      "\003\022\n\n\006UPSERT\020\004\022\t\n\005MERGE\020\005\022\r\n\tOTHER_DML\020\006\022" +
+      "\n\n\006CREATE\020\007\022\010\n\004DROP\020\010\022\t\n\005ALTER\020\t\022\r\n\tOTHE" +
+      "R_DDL\020\n\022\010\n\004CALL\020\013*\342\003\n\003Rep\022\025\n\021PRIMITIVE_B" +
+      "OOLEAN\020\000\022\022\n\016PRIMITIVE_BYTE\020\001\022\022\n\016PRIMITIV" +
+      "E_CHAR\020\002\022\023\n\017PRIMITIVE_SHORT\020\003\022\021\n\rPRIMITI" +
+      "VE_INT\020\004\022\022\n\016PRIMITIVE_LONG\020\005\022\023\n\017PRIMITIV" +
+      "E_FLOAT\020\006\022\024\n\020PRIMITIVE_DOUBLE\020\007\022\013\n\007BOOLE" +
+      "AN\020\010\022\010\n\004BYTE\020\t\022\r\n\tCHARACTER\020\n\022\t\n\005SHORT\020\013" +
+      "\022\013\n\007INTEGER\020\014\022\010\n\004LONG\020\r\022\t\n\005FLOAT\020\016\022\n\n\006DO" +
+      "UBLE\020\017\022\017\n\013BIG_INTEGER\020\031\022\017\n\013BIG_DECIMAL\020\032",
+      "\022\021\n\rJAVA_SQL_TIME\020\020\022\026\n\022JAVA_SQL_TIMESTAM" +
+      "P\020\021\022\021\n\rJAVA_SQL_DATE\020\022\022\022\n\016JAVA_UTIL_DATE" +
+      "\020\023\022\017\n\013BYTE_STRING\020\024\022\n\n\006STRING\020\025\022\n\n\006NUMBE" +
+      "R\020\026\022\n\n\006OBJECT\020\027\022\010\n\004NULL\020\030\022\t\n\005ARRAY\020\033\022\n\n\006" +
+      "STRUCT\020\034\022\014\n\010MULTISET\020\035*^\n\010Severity\022\024\n\020UN" +
+      "KNOWN_SEVERITY\020\000\022\022\n\016FATAL_SEVERITY\020\001\022\022\n\016" +
+      "ERROR_SEVERITY\020\002\022\024\n\020WARNING_SEVERITY\020\003*\327" +
+      "\004\n\021MetaDataOperation\022\022\n\016GET_ATTRIBUTES\020\000" +
+      "\022\033\n\027GET_BEST_ROW_IDENTIFIER\020\001\022\020\n\014GET_CAT" +
+      "ALOGS\020\002\022\036\n\032GET_CLIENT_INFO_PROPERTIES\020\003\022",
+      "\031\n\025GET_COLUMN_PRIVILEGES\020\004\022\017\n\013GET_COLUMN" +
+      "S\020\005\022\027\n\023GET_CROSS_REFERENCE\020\006\022\025\n\021GET_EXPO" +
+      "RTED_KEYS\020\007\022\030\n\024GET_FUNCTION_COLUMNS\020\010\022\021\n" +
+      "\rGET_FUNCTIONS\020\t\022\025\n\021GET_IMPORTED_KEYS\020\n\022" +
+      "\022\n\016GET_INDEX_INFO\020\013\022\024\n\020GET_PRIMARY_KEYS\020" +
+      "\014\022\031\n\025GET_PROCEDURE_COLUMNS\020\r\022\022\n\016GET_PROC" +
+      "EDURES\020\016\022\026\n\022GET_PSEUDO_COLUMNS\020\017\022\017\n\013GET_" +
+      "SCHEMAS\020\020\022\031\n\025GET_SCHEMAS_WITH_ARGS\020\021\022\024\n\020" +
+      "GET_SUPER_TABLES\020\022\022\023\n\017GET_SUPER_TYPES\020\023\022" +
+      "\030\n\024GET_TABLE_PRIVILEGES\020\024\022\016\n\nGET_TABLES\020",
+      "\025\022\023\n\017GET_TABLE_TYPES\020\026\022\021\n\rGET_TYPE_INFO\020" +
+      "\027\022\014\n\010GET_UDTS\020\030\022\027\n\023GET_VERSION_COLUMNS\020\031" +
+      "*\"\n\tStateType\022\007\n\003SQL\020\000\022\014\n\010METADATA\020\001B\"\n " +
+      "org.apache.calcite.avatica.protob\006proto3"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -16362,7 +17103,7 @@ package org.apache.calcite.avatica.proto;
     internal_static_ColumnValue_fieldAccessorTable = new
       com.google.protobuf.GeneratedMessage.FieldAccessorTable(
         internal_static_ColumnValue_descriptor,
-        new java.lang.String[] { "Value", });
+        new java.lang.String[] { "Value", "ArrayValue", "HasArrayValue", "ScalarValue", });
     internal_static_TypedValue_descriptor =
       getDescriptor().getMessageTypes().get(12);
     internal_static_TypedValue_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/calcite/blob/361096ba/avatica/src/main/protobuf/common.proto
----------------------------------------------------------------------
diff --git a/avatica/src/main/protobuf/common.proto b/avatica/src/main/protobuf/common.proto
index 5421ffc..bd116c3 100644
--- a/avatica/src/main/protobuf/common.proto
+++ b/avatica/src/main/protobuf/common.proto
@@ -182,7 +182,10 @@ message WireMessage {
 
 // A value might be a TypedValue or an Array of TypedValue's
 message ColumnValue {
-  repeated TypedValue value = 1;
+  repeated TypedValue value = 1; // deprecated, use array_value or scalar_value
+  repeated TypedValue array_value = 2;
+  bool has_array_value = 3; // Is an array value set?
+  TypedValue scalar_value = 4;
 }
 
 // Generic wrapper to support any SQL type. Struct-like to work around no polymorphism construct.

http://git-wip-us.apache.org/repos/asf/calcite/blob/361096ba/avatica/src/test/java/org/apache/calcite/avatica/FrameTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/FrameTest.java b/avatica/src/test/java/org/apache/calcite/avatica/FrameTest.java
index e4d524c..bdd989b 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/FrameTest.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/FrameTest.java
@@ -17,23 +17,32 @@
 package org.apache.calcite.avatica;
 
 import org.apache.calcite.avatica.Meta.Frame;
+import org.apache.calcite.avatica.proto.Common;
+import org.apache.calcite.avatica.proto.Common.ColumnValue;
+import org.apache.calcite.avatica.proto.Common.TypedValue;
 
 import org.junit.Test;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Tests serialization of {@link Frame}.
  */
 public class FrameTest {
 
+  private static final TypedValue NUMBER_VALUE = TypedValue.newBuilder().setNumberValue(1)
+      .setType(Common.Rep.LONG).build();
+
   private void serializeAndTestEquality(Frame frame) {
     Frame frameCopy = Frame.fromProto(frame.toProto());
 
@@ -94,6 +103,63 @@ public class FrameTest {
 
     serializeAndTestEquality(singleRow);
   }
+
+  @Test public void testMalformedColumnValue() {
+    // Invalid ColumnValue: has an array and scalar
+    final ColumnValue bothAttributesColumnValue = ColumnValue.newBuilder().setHasArrayValue(true)
+        .setScalarValue(NUMBER_VALUE).build();
+    // Note omission of setScalarValue(TypedValue).
+    final ColumnValue neitherAttributeColumnValue = ColumnValue.newBuilder().setHasArrayValue(false)
+        .build();
+
+    try {
+      Frame.validateColumnValue(bothAttributesColumnValue);
+      fail("Validating the ColumnValue should have failed as it has an array and scalar");
+    } catch (IllegalArgumentException e) {
+      // Pass
+    }
+
+    try {
+      Frame.validateColumnValue(neitherAttributeColumnValue);
+      fail("Validating the ColumnValue should have failed as it has neither an array nor scalar");
+    } catch (IllegalArgumentException e) {
+      // Pass
+    }
+  }
+
+  @Test public void testColumnValueBackwardsCompatibility() {
+    // 1
+    final ColumnValue oldStyleScalarValue = ColumnValue.newBuilder().addValue(NUMBER_VALUE).build();
+    // [1, 1]
+    final ColumnValue oldStyleArrayValue = ColumnValue.newBuilder().addValue(NUMBER_VALUE)
+        .addValue(NUMBER_VALUE).build();
+
+    assertFalse(Frame.isNewStyleColumn(oldStyleScalarValue));
+    assertFalse(Frame.isNewStyleColumn(oldStyleArrayValue));
+
+    Object scalar = Frame.parseOldStyleColumn(oldStyleScalarValue);
+    assertEquals(1L, scalar);
+
+    Object array = Frame.parseOldStyleColumn(oldStyleArrayValue);
+    assertEquals(Arrays.asList(1L, 1L), array);
+  }
+
+  @Test public void testColumnValueParsing() {
+    // 1
+    final ColumnValue scalarValue = ColumnValue.newBuilder().setScalarValue(NUMBER_VALUE).build();
+    // [1, 1]
+    final ColumnValue arrayValue = ColumnValue.newBuilder().addArrayValue(NUMBER_VALUE)
+        .addArrayValue(NUMBER_VALUE).setHasArrayValue(true).build();
+
+    assertTrue(Frame.isNewStyleColumn(scalarValue));
+    assertTrue(Frame.isNewStyleColumn(arrayValue));
+
+    Object scalar = Frame.parseColumn(scalarValue);
+    assertEquals(1L, scalar);
+
+    Object array = Frame.parseColumn(arrayValue);
+    assertEquals(Arrays.asList(1L, 1L), array);
+  }
 }
 
 // End FrameTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/361096ba/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
index 7d24598..a10c7dc 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
@@ -19,6 +19,7 @@ package org.apache.calcite.avatica.remote;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.Meta.Frame;
 import org.apache.calcite.avatica.proto.Common;
+import org.apache.calcite.avatica.proto.Common.ColumnValue;
 import org.apache.calcite.avatica.proto.Requests;
 import org.apache.calcite.avatica.proto.Responses;
 import org.apache.calcite.avatica.remote.Handler.HandlerResponse;
@@ -113,16 +114,20 @@ public class ProtobufHandlerTest {
     Iterator<Common.ColumnValue> iter = columnValues.iterator();
     assertTrue(iter.hasNext());
     Common.ColumnValue column = iter.next();
-    assertEquals(1, column.getValueCount());
+    assertTrue("The Column should have contained a scalar: " + column,
+        ProtobufService.hasField(column, column.getDescriptorForType(),
+            ColumnValue.SCALAR_VALUE_FIELD_NUMBER));
 
-    Common.TypedValue value = column.getValue(0);
+    Common.TypedValue value = column.getScalarValue();
     assertEquals(Common.Rep.BOOLEAN, value.getType());
     assertEquals(true, value.getBoolValue());
 
     assertTrue(iter.hasNext());
     column = iter.next();
-    assertEquals(1, column.getValueCount());
-    value = column.getValue(0);
+    assertTrue("The Column should have contained a scalar: " + column,
+        ProtobufService.hasField(column, column.getDescriptorForType(),
+            ColumnValue.SCALAR_VALUE_FIELD_NUMBER));
+    value = column.getScalarValue();
     assertEquals(Common.Rep.STRING, value.getType());
     assertEquals("my_string", value.getStringValue());
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/361096ba/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
index cfc92d9..c75bdb0 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
@@ -19,7 +19,9 @@ package org.apache.calcite.avatica.remote;
 import org.apache.calcite.avatica.AvaticaParameter;
 import org.apache.calcite.avatica.AvaticaSeverity;
 import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ColumnMetaData.ArrayType;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.ColumnMetaData.ScalarType;
 import org.apache.calcite.avatica.ConnectionPropertiesImpl;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.Meta.Frame;
@@ -71,6 +73,7 @@ import org.junit.runners.Parameterized.Parameters;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.sql.DatabaseMetaData;
 import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -252,6 +255,16 @@ public class ProtobufTranslationImplTest<T> {
     return requests;
   }
 
+  private static ColumnMetaData getArrayColumnMetaData(ScalarType componentType, int index,
+      String name) {
+    ArrayType arrayType = ColumnMetaData.array(componentType, "Array", Rep.ARRAY);
+    return new ColumnMetaData(
+        index, false, true, false, false, DatabaseMetaData.columnNullable,
+        true, -1, name, name, null,
+        0, 0, null, null, arrayType, true, false, false,
+        "ARRAY");
+  }
+
   /**
    * Generates a collection of Responses whose serialization will be tested.
    */
@@ -262,18 +275,22 @@ public class ProtobufTranslationImplTest<T> {
     // Nested classes (Signature, ColumnMetaData, CursorFactory, etc) are implicitly getting tested)
 
     // Stub out the metadata for a row
+    ScalarType arrayComponentType = ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER);
+    ColumnMetaData arrayColumnMetaData = getArrayColumnMetaData(arrayComponentType, 2, "counts");
     List<ColumnMetaData> columns =
         Arrays.asList(MetaImpl.columnMetaData("str", 0, String.class),
-            MetaImpl.columnMetaData("count", 1, Integer.class));
+            MetaImpl.columnMetaData("count", 1, Integer.class),
+            arrayColumnMetaData);
     List<AvaticaParameter> params =
         Arrays.asList(
             new AvaticaParameter(false, 10, 0, Types.VARCHAR, "VARCHAR",
                 String.class.getName(), "str"));
     Meta.CursorFactory cursorFactory = Meta.CursorFactory.create(Style.LIST, Object.class,
-        Arrays.asList("str", "count"));
+        Arrays.asList("str", "count", "counts"));
     // The row values
     List<Object> rows = new ArrayList<>();
-    rows.add(new Object[] {"str_value", 50});
+    rows.add(new Object[] {"str_value1", 50, Arrays.asList(1, 2, 3)});
+    rows.add(new Object[] {"str_value2", 100, Arrays.asList(1)});
 
     // Create the signature and frame using the metadata and values
     Signature signature = Signature.create(columns, "sql", params, cursorFactory,

http://git-wip-us.apache.org/repos/asf/calcite/blob/361096ba/site/_docs/avatica_protobuf_reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/avatica_protobuf_reference.md b/site/_docs/avatica_protobuf_reference.md
index c6b662d..02a8ab1 100644
--- a/site/_docs/avatica_protobuf_reference.md
+++ b/site/_docs/avatica_protobuf_reference.md
@@ -28,12 +28,14 @@ miscellaneous:
   - { name: "AvaticaSeverity" }
   - { name: "AvaticaType" }
   - { name: "ColumnMetaData" }
+  - { name: "ColumnValue" }
   - { name: "ConnectionProperties" }
   - { name: "CursorFactory" }
   - { name: "DatabaseProperty" }
   - { name: "Frame" }
   - { name: "QueryState" }
   - { name: "Rep" }
+  - { name: "Row" }
   - { name: "RpcMetadata" }
   - { name: "Signature" }
   - { name: "StateType" }
@@ -947,11 +949,20 @@ message Row {
 
 {% highlight protobuf %}
 message ColumnValue {
-  repeated TypedValue value = 1;
+  repeated TypedValue value = 1; // Deprecated!
+  repeated ColumnValue array_value = 2;
+  boolean has_array_value = 3;
+  TypedValue scalar_value = 4;
 }
 {% endhighlight %}
 
-`value` A collection of <a href="#typedvalue">TypedValue</a>s.
+`value` The pre Calcite-1.6 means of serializing <a href="#typedvalue">TypedValue</a>s. Not used anymore.
+
+`array_value` The value of this column if it is an array (not a scalar).
+
+`has_array_value` Should be set to true if `array_value` is set.
+
+`scalar_value` The value of this column if it is a scalar (not an array).
 
 ### QueryState
 


[06/50] [abbrv] calcite git commit: [CALCITE-861] Be explicit that "mvn test" needs to be invoked

Posted by jh...@apache.org.
[CALCITE-861] Be explicit that "mvn test" needs to be invoked

Close apache/calcite#153


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

Branch: refs/heads/branch-release
Commit: b94a00e335d0378ec912cb029f407abf811aaa12
Parents: 894dd9b
Author: Josh Elser <el...@apache.org>
Authored: Fri Oct 16 00:55:28 2015 -0400
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jan 8 11:12:04 2016 -0800

----------------------------------------------------------------------
 site/_docs/howto.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/b94a00e3/site/_docs/howto.md
----------------------------------------------------------------------
diff --git a/site/_docs/howto.md b/site/_docs/howto.md
index c741174..3a03364 100644
--- a/site/_docs/howto.md
+++ b/site/_docs/howto.md
@@ -74,6 +74,7 @@ The test suite will run by default when you build, unless you specify
 # Note: "mvn clean install" does not work; use "mvn clean" then "mvn install"
 $ mvn clean
 $ mvn -DskipTests install
+$ mvn test
 {% endhighlight %}
 
 There are other options that control which tests are run, and in what


[12/50] [abbrv] calcite git commit: Document RelOptPredicateList

Posted by jh...@apache.org.
Document RelOptPredicateList


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

Branch: refs/heads/branch-release
Commit: 970a8ca9130111d9ed3425e6ef74d9c86f43c54a
Parents: b1fdd12
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Aug 24 14:29:37 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 .../calcite/plan/RelOptPredicateList.java       | 60 ++++++++++++++++++++
 1 file changed, 60 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/970a8ca9/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java b/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
index 658ae7f..1dce56d 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptPredicateList.java
@@ -24,14 +24,57 @@ import com.google.common.collect.ImmutableList;
 /**
  * Predicates that are known to hold in the output of a particular relational
  * expression.
+ *
+ * <p><b>Pulled up predicates</b> (field {@link #pulledUpPredicates} are
+ * predicates that apply to every row output by the relational expression. They
+ * are inferred from the input relational expression(s) and the relational
+ * operator.
+ *
+ * <p>For example, if you apply {@code Filter(x > 1)} to a relational
+ * expression that has a predicate {@code y < 10} then the pulled up predicates
+ * for the Filter are {@code [y < 10, x > ]}.
+ *
+ * <p><b>Inferred predicates</b> only apply to joins. If there there is a
+ * predicate on the left input to a join, and that predicate is over columns
+ * used in the join condition, then a predicate can be inferred on the right
+ * input to the join. (And vice versa.)
+ *
+ * <p>For example, in the query
+ * <blockquote>SELECT *<br>
+ * FROM emp<br>
+ * JOIN dept ON emp.deptno = dept.deptno
+ * WHERE emp.gender = 'F' AND emp.deptno &lt; 10</blockquote>
+ * we have
+ * <ul>
+ *   <li>left: {@code Filter(Scan(EMP), deptno < 10},
+ *       predicates: {@code [deptno < 10]}
+ *   <li>right: {@code Scan(DEPT)}, predicates: {@code []}
+ *   <li>join: {@code Join(left, right, emp.deptno = dept.deptno},
+ *      leftInferredPredicates: [],
+ *      rightInferredPredicates: [deptno &lt; 10],
+ *      pulledUpPredicates: [emp.gender = 'F', emp.deptno &lt; 10,
+ *      emp.deptno = dept.deptno, dept.deptno &lt; 10]
+ * </ul>
+ *
+ * <p>Note that the predicate from the left input appears in
+ * {@code rightInferredPredicates}. Predicates from several sources appear in
+ * {@code pulledUpPredicates}.
  */
 public class RelOptPredicateList {
   private static final ImmutableList<RexNode> EMPTY_LIST = ImmutableList.of();
   public static final RelOptPredicateList EMPTY =
       new RelOptPredicateList(EMPTY_LIST, EMPTY_LIST, EMPTY_LIST);
 
+  /** Predicates that can be pulled up from the relational expression and its
+   * inputs. */
   public final ImmutableList<RexNode> pulledUpPredicates;
+
+  /** Predicates that were inferred from the right input.
+   * Empty if the relational expression is not a join. */
   public final ImmutableList<RexNode> leftInferredPredicates;
+
+  /** Predicates that were inferred from the left input.
+   * Empty if the relational expression is not a join. */
   public final ImmutableList<RexNode> rightInferredPredicates;
 
   private RelOptPredicateList(Iterable<RexNode> pulledUpPredicates,
@@ -43,6 +86,14 @@ public class RelOptPredicateList {
         ImmutableList.copyOf(rightInferredPredicates);
   }
 
+  /** Creates a RelOptPredicateList with only pulled-up predicates, no inferred
+   * predicates.
+   *
+   * <p>Use this for relational expressions other than joins.
+   *
+   * @param pulledUpPredicates Predicates that apply to the rows returned by the
+   * relational expression
+   */
   public static RelOptPredicateList of(Iterable<RexNode> pulledUpPredicates) {
     ImmutableList<RexNode> pulledUpPredicatesList =
         ImmutableList.copyOf(pulledUpPredicates);
@@ -53,6 +104,15 @@ public class RelOptPredicateList {
         EMPTY_LIST);
   }
 
+  /** Creates a RelOptPredicateList for a join.
+   *
+   * @param pulledUpPredicates Predicates that apply to the rows returned by the
+   * relational expression
+   * @param leftInferredPredicates Predicates that were inferred from the right
+   *                               input
+   * @param rightInferredPredicates Predicates that were inferred from the left
+   *                                input
+   */
   public static RelOptPredicateList of(Iterable<RexNode> pulledUpPredicates,
       Iterable<RexNode> leftInferredPredicates,
       Iterable<RexNode> rightInferredPredicates) {


[16/50] [abbrv] calcite git commit: [CALCITE-955] Litmus (continuation-passing style for methods that check invariants)

Posted by jh...@apache.org.
[CALCITE-955] Litmus (continuation-passing style for methods that check invariants)


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

Branch: refs/heads/branch-release
Commit: 9a52b5eda9fcfb4e6a7ad94fbd4a1c3fa2fb869a
Parents: b5b28f0
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Nov 5 18:09:38 2015 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     |  40 +++----
 .../calcite/plan/SubstitutionVisitor.java       |  19 ++--
 .../apache/calcite/plan/volcano/RelSubset.java  |   7 +-
 .../calcite/plan/volcano/VolcanoPlanner.java    |  38 +++----
 .../org/apache/calcite/rel/AbstractRelNode.java |  11 +-
 .../java/org/apache/calcite/rel/RelNode.java    |  16 ++-
 .../org/apache/calcite/rel/core/Aggregate.java  |  22 ++--
 .../java/org/apache/calcite/rel/core/Calc.java  |  20 ++--
 .../org/apache/calcite/rel/core/Filter.java     |  15 ++-
 .../java/org/apache/calcite/rel/core/Join.java  |  23 ++--
 .../org/apache/calcite/rel/core/Project.java    | 114 +++----------------
 .../org/apache/calcite/rel/core/Window.java     |  12 +-
 .../calcite/rel/logical/LogicalWindow.java      |   5 +-
 .../calcite/rel/rules/CalcRelSplitter.java      |   3 +-
 .../java/org/apache/calcite/rex/RexCall.java    |   3 +-
 .../java/org/apache/calcite/rex/RexChecker.java |  48 ++++----
 .../java/org/apache/calcite/rex/RexProgram.java | 113 +++++++++---------
 .../apache/calcite/rex/RexProgramBuilder.java   |  15 +--
 .../java/org/apache/calcite/rex/RexUtil.java    |  77 ++++++-------
 .../org/apache/calcite/schema/SchemaPlus.java   |  11 +-
 .../apache/calcite/sql/SqlBinaryOperator.java   |   8 +-
 .../java/org/apache/calcite/sql/SqlCall.java    |  11 +-
 .../org/apache/calcite/sql/SqlDataTypeSpec.java |  29 ++---
 .../org/apache/calcite/sql/SqlDynamicParam.java |  11 +-
 .../org/apache/calcite/sql/SqlIdentifier.java   |  14 +--
 .../calcite/sql/SqlIntervalQualifier.java       |   8 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java |  11 +-
 .../java/org/apache/calcite/sql/SqlNode.java    |  29 +++--
 .../org/apache/calcite/sql/SqlNodeList.java     |  19 ++--
 .../org/apache/calcite/sql/SqlOperator.java     |   3 +-
 .../apache/calcite/sql/SqlPostfixOperator.java  |   8 +-
 .../apache/calcite/sql/SqlPrefixOperator.java   |   8 +-
 .../java/org/apache/calcite/sql/SqlWindow.java  |   9 +-
 .../apache/calcite/sql/fun/SqlInOperator.java   |  10 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |   8 ++
 .../calcite/sql2rel/SqlToRelConverter.java      |   8 +-
 .../java/org/apache/calcite/util/Litmus.java    |  36 +++++-
 .../main/java/org/apache/calcite/util/Util.java |   2 +
 .../calcite/test/SqlToRelConverterTest.java     |   5 +-
 39 files changed, 404 insertions(+), 445 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/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 46a1dbc..b932cef 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -72,6 +72,7 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
 import org.apache.calcite.util.Util;
@@ -1625,7 +1626,7 @@ public abstract class RelOptUtil {
    * @param type1 First type
    * @param desc2 Description of second type
    * @param type2 Second type
-   * @param fail  Whether to assert if they are not equal
+   * @param litmus What to do if an error is detected (types are not equal)
    * @return Whether the types are equal
    */
   public static boolean eq(
@@ -1633,22 +1634,21 @@ public abstract class RelOptUtil {
       RelDataType type1,
       final String desc2,
       RelDataType type2,
-      boolean fail) {
+      Litmus litmus) {
     // if any one of the types is ANY return true
     if (type1.getSqlTypeName() == SqlTypeName.ANY
         || type2.getSqlTypeName() == SqlTypeName.ANY) {
-      return true;
+      return litmus.succeed();
     }
 
     if (type1 != type2) {
-      assert !fail : "type mismatch:\n"
+      return litmus.fail("type mismatch:\n"
           + desc1 + ":\n"
           + type1.getFullTypeString() + "\n"
           + desc2 + ":\n"
-          + type2.getFullTypeString();
-      return false;
+          + type2.getFullTypeString());
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**
@@ -1660,7 +1660,7 @@ public abstract class RelOptUtil {
    * @param type1 First type
    * @param desc2 Description of role of second type
    * @param type2 Second type
-   * @param fail  Whether to assert if they are not equal
+   * @param litmus Whether to assert if they are not equal
    * @return Whether the types are equal
    */
   public static boolean equal(
@@ -1668,26 +1668,22 @@ public abstract class RelOptUtil {
       RelDataType type1,
       final String desc2,
       RelDataType type2,
-      boolean fail) {
+      Litmus litmus) {
     if (!areRowTypesEqual(type1, type2, false)) {
-      if (fail) {
-        throw new AssertionError(
-            "Type mismatch:\n"
-            + desc1 + ":\n"
-            + type1.getFullTypeString() + "\n"
-            + desc2 + ":\n"
-            + type2.getFullTypeString());
-      }
-      return false;
+      return litmus.fail("Type mismatch:\n"
+          + desc1 + ":\n"
+          + type1.getFullTypeString() + "\n"
+          + desc2 + ":\n"
+          + type2.getFullTypeString());
     }
-    return true;
+    return litmus.succeed();
   }
 
   /** Returns whether two relational expressions have the same row-type. */
   public static boolean equalType(String desc0, RelNode rel0, String desc1,
-      RelNode rel1, boolean fail) {
+      RelNode rel1, Litmus litmus) {
     // TODO: change 'equal' to 'eq', which is stronger.
-    return equal(desc0, rel0.getRowType(), desc1, rel1.getRowType(), fail);
+    return equal(desc0, rel0.getRowType(), desc1, rel1.getRowType(), litmus);
   }
 
   /**
@@ -2515,7 +2511,7 @@ public abstract class RelOptUtil {
       // Short-cut common case.
       return query;
     }
-    assert equalType("find", find, "replace", replace, true);
+    assert equalType("find", find, "replace", replace, Litmus.THROW);
     if (query == find) {
       // Short-cut another common case.
       return replace;

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/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 c88f51d..8dd0d01 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -54,6 +54,7 @@ import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mapping;
@@ -405,7 +406,7 @@ public class SubstitutionVisitor {
     assert false; // not called
     MutableRel replacement = toMutable(replacement_);
     assert MutableRels.equalType(
-        "target", target, "replacement", replacement, true);
+        "target", target, "replacement", replacement, Litmus.THROW);
     replacementMap.put(target, replacement);
     final UnifyResult unifyResult = matchRecurse(target);
     if (unifyResult == null) {
@@ -458,7 +459,7 @@ public class SubstitutionVisitor {
    */
   private List<List<Replacement>> go(MutableRel replacement) {
     assert MutableRels.equalType(
-        "target", target, "replacement", replacement, true);
+        "target", target, "replacement", replacement, Litmus.THROW);
     final List<MutableRel> queryDescendants = MutableRels.descendants(query);
     final List<MutableRel> targetDescendants = MutableRels.descendants(target);
 
@@ -895,7 +896,8 @@ public class SubstitutionVisitor {
 
     public UnifyResult result(MutableRel result) {
       assert MutableRels.contains(result, target);
-      assert MutableRels.equalType("result", result, "query", query, true);
+      assert MutableRels.equalType("result", result, "query", query,
+          Litmus.THROW);
       MutableRel replace = replacementMap.get(target);
       if (replace != null) {
         assert false; // replacementMap is always empty
@@ -931,7 +933,8 @@ public class SubstitutionVisitor {
 
     UnifyResult(UnifyRuleCall call, MutableRel result) {
       this.call = call;
-      assert MutableRels.equalType("query", call.query, "result", result, true);
+      assert MutableRels.equalType("query", call.query, "result", result,
+          Litmus.THROW);
       this.result = result;
     }
   }
@@ -1661,7 +1664,7 @@ public class SubstitutionVisitor {
         List<RexNode> projects) {
       super(MutableRelType.PROJECT, rowType, input);
       this.projects = projects;
-      assert RexUtil.compatibleTypes(projects, rowType, true);
+      assert RexUtil.compatibleTypes(projects, rowType, Litmus.THROW);
     }
 
     public static MutableProject of(RelDataType rowType, MutableRel input,
@@ -2096,9 +2099,9 @@ public class SubstitutionVisitor {
 
     /** Returns whether two relational expressions have the same row-type. */
     public static boolean equalType(String desc0, MutableRel rel0, String desc1,
-        MutableRel rel1, boolean fail) {
+        MutableRel rel1, Litmus litmus) {
       return RelOptUtil.equal(desc0, rel0.getRowType(),
-          desc1, rel1.getRowType(), fail);
+          desc1, rel1.getRowType(), litmus);
     }
 
     /** Within a relational expression {@code query}, replaces occurrences of
@@ -2112,7 +2115,7 @@ public class SubstitutionVisitor {
         // Short-cut common case.
         return null;
       }
-      assert equalType("find", find, "replace", replace, true);
+      assert equalType("find", find, "replace", replace, Litmus.THROW);
       return replaceRecurse(query, find, replace);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
index 0ecc264..bb42218 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
@@ -271,10 +272,8 @@ public class RelSubset extends AbstractRelNode {
     // If this isn't the first rel in the set, it must have compatible
     // row type.
     if (set.rel != null) {
-      if (!RelOptUtil.equal("rowtype of new rel", rel.getRowType(),
-          "rowtype of set", getRowType(), true)) {
-        throw new AssertionError();
-      }
+      RelOptUtil.equal("rowtype of new rel", rel.getRowType(),
+          "rowtype of set", getRowType(), Litmus.THROW);
     }
     set.addInternal(rel);
     Set<String> variablesSet = RelOptUtil.getVariablesSet(rel);

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
index 1389a36..4186232 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
@@ -66,6 +66,7 @@ import org.apache.calcite.rel.rules.UnionToDistinctRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.SaffronProperties;
 import org.apache.calcite.util.Util;
@@ -168,7 +169,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   /**
    * List of all sets. Used only for debugging.
    */
-  final List<RelSet> allSets = new ArrayList<RelSet>();
+  final List<RelSet> allSets = new ArrayList<>();
 
   /**
    * Canonical map from {@link String digest} to the unique
@@ -180,7 +181,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * null VARCHAR(10).
    */
   private final Map<Pair<String, RelDataType>, RelNode> mapDigestToRel =
-      new HashMap<Pair<String, RelDataType>, RelNode>();
+      new HashMap<>();
 
   /**
    * Map each registered expression ({@link RelNode}) to its equivalence set
@@ -193,7 +194,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * to be careful, otherwise it gets incestuous.</p>
    */
   private final IdentityHashMap<RelNode, RelSubset> mapRel2Subset =
-      new IdentityHashMap<RelNode, RelSubset>();
+      new IdentityHashMap<>();
 
   /**
    * The importance of relational expressions.
@@ -205,13 +206,12 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * <p>If a RelNode has 0 importance, all {@link RelOptRuleCall}s using it
    * are ignored, and future RelOptRuleCalls are not queued up.
    */
-  final Map<RelNode, Double> relImportances = new HashMap<RelNode, Double>();
+  final Map<RelNode, Double> relImportances = new HashMap<>();
 
   /**
    * List of all schemas which have been registered.
    */
-  private final Set<RelOptSchema> registeredSchemas =
-      new HashSet<RelOptSchema>();
+  private final Set<RelOptSchema> registeredSchemas = new HashSet<>();
 
   /**
    * Holds rule calls waiting to be fired.
@@ -221,12 +221,12 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   /**
    * Holds the currently registered RelTraitDefs.
    */
-  private final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
+  private final List<RelTraitDef> traitDefs = new ArrayList<>();
 
   /**
    * Set of all registered rules.
    */
-  protected final Set<RelOptRule> ruleSet = new HashSet<RelOptRule>();
+  protected final Set<RelOptRule> ruleSet = new HashSet<>();
 
   private int nextSetId = 0;
 
@@ -261,11 +261,9 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   private final Map<List<String>, RelOptLattice> latticeByName =
       Maps.newLinkedHashMap();
 
-  final Map<RelNode, Provenance> provenanceMap =
-      new HashMap<RelNode, Provenance>();
+  final Map<RelNode, Provenance> provenanceMap = new HashMap<>();
 
-  private final List<VolcanoRuleCall> ruleCallStack =
-      new ArrayList<VolcanoRuleCall>();
+  private final List<VolcanoRuleCall> ruleCallStack = new ArrayList<>();
 
   /** Zero cost, according to {@link #costFactory}. Not necessarily a
    * {@link org.apache.calcite.plan.volcano.VolcanoCost}. */
@@ -533,7 +531,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   }
 
   private static Set<RelOptTable> findTables(RelNode rel) {
-    final Set<RelOptTable> usedTables = new LinkedHashSet<RelOptTable>();
+    final Set<RelOptTable> usedTables = new LinkedHashSet<>();
     new RelVisitor() {
       @Override public void visit(RelNode node, int ordinal, RelNode parent) {
         if (node instanceof TableScan) {
@@ -896,7 +894,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   private String provenance(RelNode root) {
     final StringWriter sw = new StringWriter();
     final PrintWriter pw = new PrintWriter(sw);
-    final List<RelNode> nodes = new ArrayList<RelNode>();
+    final List<RelNode> nodes = new ArrayList<>();
     new RelVisitor() {
       public void visit(RelNode node, int ordinal, RelNode parent) {
         nodes.add(node);
@@ -904,7 +902,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
       }
       // CHECKSTYLE: IGNORE 1
     }.go(root);
-    final Set<RelNode> visited = new HashSet<RelNode>();
+    final Set<RelNode> visited = new HashSet<>();
     for (RelNode node : nodes) {
       provenanceRecurse(pw, node, 0, visited);
     }
@@ -953,7 +951,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
         new RelVisitor() {
           int depth = 0;
 
-          HashSet<RelSubset> visitedSubsets = new HashSet<RelSubset>();
+          final HashSet<RelSubset> visitedSubsets = new HashSet<>();
 
           public void visit(
               RelNode p,
@@ -993,7 +991,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * {@link Convention#NONE} and boosts their importance by 25%.
    */
   private void injectImportanceBoost() {
-    HashSet<RelSubset> requireBoost = new HashSet<RelSubset>();
+    final HashSet<RelSubset> requireBoost = new HashSet<>();
 
   SUBSET_LOOP:
     for (RelSubset subset : ruleQueue.subsetImportances.keySet()) {
@@ -1031,7 +1029,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
           rel.getRowType(),
           "equivRel rowtype",
           equivRel.getRowType(),
-          true);
+          Litmus.THROW);
       set = getSet(equivRel);
     }
     final RelSubset subset = registerImpl(rel, set);
@@ -1688,7 +1686,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
           rel,
           new RuleProvenance(
               ruleCall.rule,
-              ImmutableList.<RelNode>copyOf(ruleCall.rels),
+              ImmutableList.copyOf(ruleCall.rels),
               ruleCall.id));
     }
 
@@ -1704,7 +1702,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
       assert RelOptUtil.equal(
           "left", equivExp.getRowType(),
           "right", rel.getRowType(),
-          true);
+          Litmus.THROW);
       RelSet equivSet = getSet(equivExp);
       if (equivSet != null) {
         if (LOGGER.isLoggable(Level.FINER)) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
index 0e7facd..1938447 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -34,6 +34,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
@@ -196,8 +197,12 @@ public abstract class AbstractRelNode implements RelNode {
     return className;
   }
 
+  public boolean isValid(Litmus litmus) {
+    return litmus.succeed();
+  }
+
   public boolean isValid(boolean fail) {
-    return true;
+    return isValid(Litmus.THROW);
   }
 
   /** @deprecated Use {@link RelMetadataQuery#collations(RelNode)} */
@@ -310,7 +315,7 @@ public abstract class AbstractRelNode implements RelNode {
             input.getRowType(),
             "rowtype of rel after registration",
             e.getRowType(),
-            true);
+            Litmus.THROW);
       }
       inputs.add(e);
     }
@@ -319,7 +324,7 @@ public abstract class AbstractRelNode implements RelNode {
       r = copy(getTraitSet(), inputs);
     }
     r.recomputeDigest();
-    assert r.isValid(true);
+    assert r.isValid(Litmus.THROW);
     return r;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/RelNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelNode.java b/core/src/main/java/org/apache/calcite/rel/RelNode.java
index 7cf4de7..29dc023 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelNode.java
@@ -29,6 +29,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 
 import java.util.List;
 import java.util.Set;
@@ -301,20 +302,23 @@ public interface RelNode extends RelOptNode, Cloneable {
    * Returns whether this relational expression is valid.
    *
    * <p>If assertions are enabled, this method is typically called with <code>
-   * fail</code> = <code>true</code>, as follows:
+   * litmus</code> = <code>THROW</code>, as follows:
    *
    * <blockquote>
-   * <pre>assert rel.isValid(true)</pre>
+   * <pre>assert rel.isValid(Litmus.THROW)</pre>
    * </blockquote>
    *
-   * This signals that the method can throw an {@link AssertionError} if it is
-   * not valid.
+   * <p>This signals that the method can throw an {@link AssertionError} if it
+   * is not valid.
    *
-   * @param fail Whether to fail if invalid
+   * @param litmus What to do if invalid
    * @return Whether relational expression is valid
    * @throws AssertionError if this relational expression is invalid and
-   *                        fail=true and assertions are enabled
+   *                        litmus is THROW
    */
+  boolean isValid(Litmus litmus);
+
+  @Deprecated // to be removed before 2.0
   boolean isValid(boolean fail);
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/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 1a46ffd..1a07bed 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
@@ -40,6 +40,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidatorException;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -139,7 +140,7 @@ public abstract class Aggregate extends SingleRel {
     }
     assert groupSet.length() <= child.getRowType().getFieldCount();
     for (AggregateCall aggCall : aggCalls) {
-      assert typeMatchesInferred(aggCall, true);
+      assert typeMatchesInferred(aggCall, Litmus.THROW);
       Preconditions.checkArgument(aggCall.filterArg < 0
           || isPredicate(child, aggCall.filterArg),
           "filter must be BOOLEAN NOT NULL");
@@ -371,16 +372,9 @@ public abstract class Aggregate extends SingleRel {
     return builder.build();
   }
 
-  public boolean isValid(boolean fail) {
-    if (!super.isValid(fail)) {
-      assert !fail;
-      return false;
-    }
-    if (!Util.isDistinct(getRowType().getFieldNames())) {
-      assert !fail : getRowType();
-      return false;
-    }
-    return true;
+  public boolean isValid(Litmus litmus) {
+    return super.isValid(litmus)
+        && litmus.check(Util.isDistinct(getRowType().getFieldNames()), getRowType());
   }
 
   /**
@@ -388,12 +382,12 @@ public abstract class Aggregate extends SingleRel {
    * type it was given when it was created.
    *
    * @param aggCall Aggregate call
-   * @param fail    Whether to fail if the types do not match
+   * @param litmus What to do if an error is detected (types do not match)
    * @return Whether the inferred and declared types match
    */
   private boolean typeMatchesInferred(
       final AggregateCall aggCall,
-      final boolean fail) {
+      final Litmus litmus) {
     SqlAggFunction aggFunction = aggCall.getAggregation();
     AggCallBinding callBinding = aggCall.createBinding(this);
     RelDataType type = aggFunction.inferReturnType(callBinding);
@@ -402,7 +396,7 @@ public abstract class Aggregate extends SingleRel {
         expectedType,
         "inferred type",
         type,
-        fail);
+        litmus);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Calc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Calc.java b/core/src/main/java/org/apache/calcite/rel/core/Calc.java
index ea2f7d1..f4067eb 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Calc.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Calc.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.util.List;
@@ -63,7 +64,7 @@ public abstract class Calc extends SingleRel {
     super(cluster, traits, child);
     this.rowType = program.getOutputRowType();
     this.program = program;
-    assert isValid(true);
+    assert isValid(Litmus.THROW);
   }
 
   @Deprecated // to be removed before 2.0
@@ -109,22 +110,21 @@ public abstract class Calc extends SingleRel {
     return copy(traitSet, child, program);
   }
 
-  public boolean isValid(boolean fail) {
+  public boolean isValid(Litmus litmus) {
     if (!RelOptUtil.equal(
         "program's input type",
         program.getInputRowType(),
         "child's output type",
-        getInput().getRowType(),
-        fail)) {
-      return false;
+        getInput().getRowType(), litmus)) {
+      return litmus.fail(null);
     }
-    if (!program.isValid(fail)) {
-      return false;
+    if (!program.isValid(litmus)) {
+      return litmus.fail(null);
     }
-    if (!program.isNormalized(fail, getCluster().getRexBuilder())) {
-      return false;
+    if (!program.isNormalized(litmus, getCluster().getRexBuilder())) {
+      return litmus.fail(null);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public RexProgram getProgram() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Filter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Filter.java b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
index 0dc3009..e5461e8 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Filter.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.Litmus;
 
 import com.google.common.collect.ImmutableList;
 
@@ -73,7 +74,7 @@ public abstract class Filter extends SingleRel {
     assert RexUtil.isFlat(condition) : condition;
     this.condition = condition;
     // Too expensive for everyday use:
-    assert !CalcitePrepareImpl.DEBUG || isValid(true);
+    assert !CalcitePrepareImpl.DEBUG || isValid(Litmus.THROW);
   }
 
   /**
@@ -110,18 +111,16 @@ public abstract class Filter extends SingleRel {
     return condition;
   }
 
-  @Override public boolean isValid(boolean fail) {
+  @Override public boolean isValid(Litmus litmus) {
     if (RexUtil.isNullabilityCast(getCluster().getTypeFactory(), condition)) {
-      assert !fail : "Cast for just nullability not allowed";
-      return false;
+      return litmus.fail("Cast for just nullability not allowed");
     }
-    final RexChecker checker = new RexChecker(getInput().getRowType(), fail);
+    final RexChecker checker = new RexChecker(getInput().getRowType(), litmus);
     condition.accept(checker);
     if (checker.getFailureCount() > 0) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public RelOptCost computeSelfCost(RelOptPlanner planner) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Join.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Join.java b/core/src/main/java/org/apache/calcite/rel/core/Join.java
index 5a0990d..bb460ff 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Join.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Join.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexChecker;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -117,22 +118,20 @@ public abstract class Join extends BiRel {
   }
 
   // TODO: enable
-  public boolean isValid_(boolean fail) {
-    if (!super.isValid(fail)) {
+  public boolean isValid_(Litmus litmus) {
+    if (!super.isValid(litmus)) {
       return false;
     }
     if (getRowType().getFieldCount()
         != getSystemFieldList().size()
         + left.getRowType().getFieldCount()
         + right.getRowType().getFieldCount()) {
-      assert !fail : "field count mismatch";
-      return false;
+      return litmus.fail("field count mismatch");
     }
     if (condition != null) {
       if (condition.getType().getSqlTypeName() != SqlTypeName.BOOLEAN) {
-        assert !fail
-            : "condition must be boolean: " + condition.getType();
-        return false;
+        return litmus.fail("condition must be boolean: "
+            + condition.getType());
       }
       // The input to the condition is a row type consisting of system
       // fields, left fields, and right fields. Very similar to the
@@ -145,16 +144,14 @@ public abstract class Join extends BiRel {
                   .addAll(getLeft().getRowType().getFieldList())
                   .addAll(getRight().getRowType().getFieldList())
                   .build(),
-              fail);
+              litmus);
       condition.accept(checker);
       if (checker.getFailureCount() > 0) {
-        assert !fail
-            : checker.getFailureCount() + " failures in condition "
-            + condition;
-        return false;
+        return litmus.fail(checker.getFailureCount()
+            + " failures in condition " + condition);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Project.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Project.java b/core/src/main/java/org/apache/calcite/rel/core/Project.java
index 09c2ec9..86f4147 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Project.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Project.java
@@ -20,7 +20,6 @@ import org.apache.calcite.linq4j.Ord;
 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.RelInput;
 import org.apache.calcite.rel.RelNode;
@@ -30,14 +29,12 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexChecker;
-import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
 import org.apache.calcite.util.Util;
@@ -82,7 +79,7 @@ public abstract class Project extends SingleRel {
     assert rowType != null;
     this.exps = ImmutableList.copyOf(projects);
     this.rowType = rowType;
-    assert isValid(true);
+    assert isValid(Litmus.THROW);
   }
 
   @Deprecated // to be removed before 2.0
@@ -174,31 +171,25 @@ public abstract class Project extends SingleRel {
     return 1;
   }
 
-  public boolean isValid(boolean fail) {
-    if (!super.isValid(fail)) {
-      assert !fail;
-      return false;
+  public boolean isValid(Litmus litmus) {
+    if (!super.isValid(litmus)) {
+      return litmus.fail(null);
     }
-    if (!RexUtil.compatibleTypes(
-        exps,
-        getRowType(),
-        true)) {
-      assert !fail;
-      return false;
+    if (!RexUtil.compatibleTypes(exps, getRowType(), litmus)) {
+      return litmus.fail("incompatible types");
     }
     RexChecker checker =
         new RexChecker(
-            getInput().getRowType(), fail);
+            getInput().getRowType(), litmus);
     for (RexNode exp : exps) {
       exp.accept(checker);
-    }
-    if (checker.getFailureCount() > 0) {
-      assert !fail;
-      return false;
+      if (checker.getFailureCount() > 0) {
+        return litmus.fail(checker.getFailureCount()
+            + " failures in expression " + exp);
+      }
     }
     if (!Util.isDistinct(rowType.getFieldNames())) {
-      assert !fail : rowType;
-      return false;
+      return litmus.fail("field names not distinct: " + rowType);
     }
     //CHECKSTYLE: IGNORE 1
     if (false && !Util.isDistinct(
@@ -214,10 +205,9 @@ public abstract class Project extends SingleRel {
       // because we need to allow
       //
       //  SELECT a, b FROM c UNION SELECT x, x FROM z
-      assert !fail : exps;
-      return false;
+      return litmus.fail("duplicate expressions: " + exps);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
@@ -374,80 +364,6 @@ public abstract class Project extends SingleRel {
     public static final int BOXED = 1;
     public static final int NONE = 0;
   }
-
-  /**
-   * Visitor which walks over a program and checks validity.
-   */
-  private static class Checker extends RexVisitorImpl<Boolean> {
-    private final boolean fail;
-    private final RelDataType inputRowType;
-    int failCount = 0;
-
-    /**
-     * Creates a Checker.
-     *
-     * @param inputRowType Input row type to expressions
-     * @param fail         Whether to throw if checker finds an error
-     */
-    private Checker(RelDataType inputRowType, boolean fail) {
-      super(true);
-      this.fail = fail;
-      this.inputRowType = inputRowType;
-    }
-
-    public Boolean visitInputRef(RexInputRef inputRef) {
-      final int index = inputRef.getIndex();
-      final List<RelDataTypeField> fields = inputRowType.getFieldList();
-      if ((index < 0) || (index >= fields.size())) {
-        assert !fail;
-        ++failCount;
-        return false;
-      }
-      if (!RelOptUtil.eq("inputRef",
-          inputRef.getType(),
-          "underlying field",
-          fields.get(index).getType(),
-          fail)) {
-        assert !fail;
-        ++failCount;
-        return false;
-      }
-      return true;
-    }
-
-    public Boolean visitLocalRef(RexLocalRef localRef) {
-      assert !fail : "localRef invalid in project";
-      ++failCount;
-      return false;
-    }
-
-    public Boolean visitFieldAccess(RexFieldAccess fieldAccess) {
-      super.visitFieldAccess(fieldAccess);
-      final RelDataType refType =
-          fieldAccess.getReferenceExpr().getType();
-      assert refType.isStruct();
-      final RelDataTypeField field = fieldAccess.getField();
-      final int index = field.getIndex();
-      if ((index < 0) || (index > refType.getFieldList().size())) {
-        assert !fail;
-        ++failCount;
-        return false;
-      }
-      final RelDataTypeField typeField =
-          refType.getFieldList().get(index);
-      if (!RelOptUtil.eq(
-          "type1",
-          typeField.getType(),
-          "type2",
-          fieldAccess.getType(),
-          fail)) {
-        assert !fail;
-        ++failCount;
-        return false;
-      }
-      return true;
-    }
-  }
 }
 
 // End Project.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Window.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Window.java b/core/src/main/java/org/apache/calcite/rel/core/Window.java
index 8db59cb..7efb115 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Window.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Window.java
@@ -40,6 +40,7 @@ import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -84,7 +85,7 @@ public abstract class Window extends SingleRel {
     this.groups = ImmutableList.copyOf(groups);
   }
 
-  @Override public boolean isValid(boolean fail) {
+  @Override public boolean isValid(Litmus litmus) {
     // In the window specifications, an aggregate call such as
     // 'SUM(RexInputRef #10)' refers to expression #10 of inputProgram.
     // (Not its projections.)
@@ -106,21 +107,20 @@ public abstract class Window extends SingleRel {
         };
 
     final RexChecker checker =
-        new RexChecker(inputTypes, fail);
+        new RexChecker(inputTypes, litmus);
     int count = 0;
     for (Group group : groups) {
       for (RexWinAggCall over : group.aggCalls) {
         ++count;
         if (!checker.isValid(over)) {
-          return false;
+          return litmus.fail(null);
         }
       }
     }
     if (count == 0) {
-      assert !fail : "empty";
-      return false;
+      return litmus.fail("empty");
     }
-    return true;
+    return litmus.succeed();
   }
 
   public RelWriter explainTerms(RelWriter pw) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
index 559dc0e..783a649 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
@@ -36,6 +36,7 @@ import org.apache.calcite.rex.RexWindow;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Objects;
@@ -222,7 +223,7 @@ public final class LogicalWindow extends Window {
                 over.getType(),
                 "aggCall",
                 aggCall.getType(),
-                true);
+                Litmus.THROW);
 
             // Find the index of the aggCall among all partitions of all
             // groups.
@@ -237,7 +238,7 @@ public final class LogicalWindow extends Window {
                 over.getType(),
                 "intermed",
                 intermediateRowType.getFieldList().get(index).getType(),
-                true);
+                Litmus.THROW);
             return new RexInputRef(
                 index,
                 over.getType());

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java b/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
index 047e950..2e4e9e0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
@@ -37,6 +37,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.graph.DefaultDirectedGraph;
 import org.apache.calcite.util.graph.DefaultEdge;
@@ -121,7 +122,7 @@ public abstract class CalcRelSplitter {
     // expression is trivial (either an atom, or a function applied to
     // references to atoms) and every expression depends only on
     // expressions to the left.
-    assert program.isValid(true);
+    assert program.isValid(Litmus.THROW);
     final List<RexNode> exprList = program.getExprList();
     final RexNode[] exprs = exprList.toArray(new RexNode[exprList.size()]);
     assert !RexUtil.containComplexExprs(exprList);

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/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 b06ffc9..8271005 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -20,6 +20,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.util.Litmus;
 
 import com.google.common.collect.ImmutableList;
 
@@ -63,7 +64,7 @@ public class RexCall extends RexNode {
     assert op.getKind() != null : op;
     this.digest = computeDigest(true);
 
-    assert op.validRexOperands(operands.size(), true) : this;
+    assert op.validRexOperands(operands.size(), Litmus.THROW) : this;
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rex/RexChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexChecker.java b/core/src/main/java/org/apache/calcite/rex/RexChecker.java
index 77edc1f..c51d923 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexChecker.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexChecker.java
@@ -19,6 +19,7 @@ package org.apache.calcite.rex;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.util.Litmus;
 
 import java.util.List;
 
@@ -55,7 +56,7 @@ import java.util.List;
 public class RexChecker extends RexVisitorImpl<Boolean> {
   //~ Instance fields --------------------------------------------------------
 
-  protected final boolean fail;
+  protected final Litmus litmus;
   protected final List<RelDataType> inputTypeList;
   protected int failCount;
 
@@ -71,11 +72,10 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
    * <p>Otherwise, each method returns whether its part of the tree is valid.
    *
    * @param inputRowType Input row type
-   * @param fail Whether to throw an {@link AssertionError} if an
-   *                     invalid node is detected
+   * @param litmus What to do if an invalid node is detected
    */
-  public RexChecker(final RelDataType inputRowType, boolean fail) {
-    this(RelOptUtil.getFieldTypeList(inputRowType), fail);
+  public RexChecker(final RelDataType inputRowType, Litmus litmus) {
+    this(RelOptUtil.getFieldTypeList(inputRowType), litmus);
   }
 
   /**
@@ -88,13 +88,12 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
    * <p>Otherwise, each method returns whether its part of the tree is valid.
    *
    * @param inputTypeList Input row type
-   * @param fail Whether to throw an {@link AssertionError} if an
-   *                      invalid node is detected
+   * @param litmus What to do if an error is detected
    */
-  public RexChecker(List<RelDataType> inputTypeList, boolean fail) {
+  public RexChecker(List<RelDataType> inputTypeList, Litmus litmus) {
     super(true);
     this.inputTypeList = inputTypeList;
-    this.fail = fail;
+    this.litmus = litmus;
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -111,36 +110,32 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
   public Boolean visitInputRef(RexInputRef ref) {
     final int index = ref.getIndex();
     if ((index < 0) || (index >= inputTypeList.size())) {
-      assert !fail
-          : "RexInputRef index " + index
-          + " out of range 0.." + (inputTypeList.size() - 1);
       ++failCount;
-      return false;
+      return litmus.fail("RexInputRef index " + index
+          + " out of range 0.." + (inputTypeList.size() - 1));
     }
     if (!ref.getType().isStruct()
         && !RelOptUtil.eq("ref", ref.getType(), "input",
-            inputTypeList.get(index), fail)) {
-      assert !fail;
+            inputTypeList.get(index), litmus)) {
       ++failCount;
-      return false;
+      return litmus.fail(null);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public Boolean visitLocalRef(RexLocalRef ref) {
-    assert !fail : "RexLocalRef illegal outside program";
     ++failCount;
-    return false;
+    return litmus.fail("RexLocalRef illegal outside program");
   }
 
   public Boolean visitCall(RexCall call) {
     for (RexNode operand : call.getOperands()) {
       Boolean valid = operand.accept(this);
       if (valid != null && !valid) {
-        return false;
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   public Boolean visitFieldAccess(RexFieldAccess fieldAccess) {
@@ -150,22 +145,19 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
     final RelDataTypeField field = fieldAccess.getField();
     final int index = field.getIndex();
     if ((index < 0) || (index > refType.getFieldList().size())) {
-      assert !fail;
       ++failCount;
-      return false;
+      return litmus.fail(null);
     }
     final RelDataTypeField typeField = refType.getFieldList().get(index);
     if (!RelOptUtil.eq(
         "type1",
         typeField.getType(),
         "type2",
-        fieldAccess.getType(),
-        fail)) {
-      assert !fail;
+        fieldAccess.getType(), litmus)) {
       ++failCount;
-      return false;
+      return litmus.fail(null);
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rex/RexProgram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgram.java b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
index 6fd12ce..58cdc40 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgram.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
 
@@ -110,7 +111,7 @@ public class RexProgram {
     this.projects = ImmutableList.copyOf(projects);
     this.condition = condition;
     this.outputRowType = outputRowType;
-    assert isValid(true);
+    assert isValid(Litmus.THROW);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -394,64 +395,53 @@ public class RexProgram {
    * will throw an {@link AssertionError} if assertions are enabled. If <code>
    * fail</code> is false, merely returns whether the program is valid.
    *
-   * @param fail Whether to fail
+   * @param litmus What to do if an error is detected
    * @return Whether the program is valid
-   * @throws AssertionError if program is invalid and <code>fail</code> is
-   *                        true and assertions are enabled
    */
-  public boolean isValid(boolean fail) {
+  public boolean isValid(Litmus litmus) {
     if (inputRowType == null) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
     if (exprs == null) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
     if (projects == null) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
     if (outputRowType == null) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
 
     // If the input row type is a struct (contains fields) then the leading
     // expressions must be references to those fields. But we don't require
     // this if the input row type is, say, a java class.
     if (inputRowType.isStruct()) {
-      if (!RexUtil.containIdentity(exprs, inputRowType, fail)) {
-        assert !fail;
-        return false;
+      if (!RexUtil.containIdentity(exprs, inputRowType, litmus)) {
+        return litmus.fail(null);
       }
 
       // None of the other fields should be inputRefs.
       for (int i = inputRowType.getFieldCount(); i < exprs.size(); i++) {
         RexNode expr = exprs.get(i);
         if (expr instanceof RexInputRef) {
-          assert !fail;
-          return false;
+          return litmus.fail(null);
         }
       }
     }
     // todo: enable
     // CHECKSTYLE: IGNORE 1
-    if (false && RexUtil.containCommonExprs(exprs, fail)) {
-      assert !fail;
-      return false;
+    if (false && RexUtil.containNoCommonExprs(exprs, litmus)) {
+      return litmus.fail(null);
     }
-    if (RexUtil.containForwardRefs(exprs, inputRowType, fail)) {
-      assert !fail;
-      return false;
+    if (!RexUtil.containNoForwardRefs(exprs, inputRowType, litmus)) {
+      return litmus.fail(null);
     }
-    if (RexUtil.containNonTrivialAggs(exprs, fail)) {
-      assert !fail;
-      return false;
+    if (!RexUtil.containNoNonTrivialAggs(exprs, litmus)) {
+      return litmus.fail(null);
     }
     final Checker checker =
         new Checker(
-            fail,
+            litmus,
             inputRowType,
             new AbstractList<RelDataType>() {
               public RelDataType get(int index) {
@@ -465,30 +455,26 @@ public class RexProgram {
             });
     if (condition != null) {
       if (!SqlTypeUtil.inBooleanFamily(condition.getType())) {
-        assert !fail : "condition must be boolean";
-        return false;
+        return litmus.fail("condition must be boolean");
       }
       condition.accept(checker);
       if (checker.failCount > 0) {
-        assert !fail;
-        return false;
+        return litmus.fail(null);
       }
     }
     for (int i = 0; i < projects.size(); i++) {
       projects.get(i).accept(checker);
       if (checker.failCount > 0) {
-        assert !fail;
-        return false;
+        return litmus.fail(null);
       }
     }
     for (int i = 0; i < exprs.size(); i++) {
       exprs.get(i).accept(checker);
       if (checker.failCount > 0) {
-        assert !fail;
-        return false;
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**
@@ -773,24 +759,39 @@ public class RexProgram {
   /**
    * Returns whether this program is in canonical form.
    *
-   * @param fail       Whether to throw an assertion error if not in canonical
-   *                   form
+   * @param litmus     What to do if an error is detected (program is not in
+   *                   canonical form)
    * @param rexBuilder Rex builder
    * @return whether in canonical form
    */
-  public boolean isNormalized(boolean fail, RexBuilder rexBuilder) {
-    final RexProgram normalizedProgram =
-        RexProgramBuilder.normalize(rexBuilder, this);
+  public boolean isNormalized(Litmus litmus, RexBuilder rexBuilder) {
+    final RexProgram normalizedProgram = normalize(rexBuilder, false);
     String normalized = normalizedProgram.toString();
     String string = toString();
     if (!normalized.equals(string)) {
-      assert !fail
-          : "Program is not normalized:\n"
+      return litmus.fail("Program is not normalized:\n"
           + "program:    " + string + "\n"
-          + "normalized: " + normalized + "\n";
-      return false;
+          + "normalized: " + normalized + "\n");
     }
-    return true;
+    return litmus.succeed();
+  }
+
+  /**
+   * Creates a simplified/normalized copy of this program.
+   *
+   * @param rexBuilder Rex builder
+   * @param simplify Whether to simplify (in addition to normalizing)
+   * @return Normalized program
+   */
+  public RexProgram normalize(RexBuilder rexBuilder, boolean simplify) {
+    // Normalize program by creating program builder from the program, then
+    // converting to a program. getProgram does not need to normalize
+    // because the builder was normalized on creation.
+    assert isValid(Litmus.THROW);
+    final RexProgramBuilder builder =
+        RexProgramBuilder.create(rexBuilder, inputRowType, exprs, projects,
+            condition, outputRowType, simplify);
+    return builder.getProgram(false);
   }
 
   //~ Inner Classes ----------------------------------------------------------
@@ -804,15 +805,14 @@ public class RexProgram {
     /**
      * Creates a Checker.
      *
-     * @param fail                 Whether to fail
+     * @param litmus               Whether to fail
      * @param inputRowType         Types of the input fields
      * @param internalExprTypeList Types of the internal expressions
      */
-    public Checker(
-        boolean fail,
+    public Checker(Litmus litmus,
         RelDataType inputRowType,
         List<RelDataType> internalExprTypeList) {
-      super(inputRowType, fail);
+      super(inputRowType, litmus);
       this.internalExprTypeList = internalExprTypeList;
     }
 
@@ -821,21 +821,18 @@ public class RexProgram {
     public Boolean visitLocalRef(RexLocalRef localRef) {
       final int index = localRef.getIndex();
       if ((index < 0) || (index >= internalExprTypeList.size())) {
-        assert !fail;
         ++failCount;
-        return false;
+        return litmus.fail(null);
       }
       if (!RelOptUtil.eq(
           "type1",
           localRef.getType(),
           "type2",
-          internalExprTypeList.get(index),
-          fail)) {
-        assert !fail;
+          internalExprTypeList.get(index), litmus)) {
         ++failCount;
-        return false;
+        return litmus.fail(null);
       }
-      return true;
+      return litmus.succeed();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
index bc89347..c292e56 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
@@ -20,6 +20,7 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -449,7 +450,7 @@ public class RexProgramBuilder {
       RexProgram program,
       RexBuilder rexBuilder,
       boolean normalize) {
-    assert program.isValid(true);
+    assert program.isValid(Litmus.THROW);
     final RelDataType inputRowType = program.getInputRowType();
     final List<RexLocalRef> projectRefs = program.getProjectList();
     final RexLocalRef conditionRef = program.getCondition();
@@ -726,8 +727,8 @@ public class RexProgramBuilder {
       boolean normalize) {
     // Initialize a program builder with the same expressions, outputs
     // and condition as the bottom program.
-    assert bottomProgram.isValid(true);
-    assert topProgram.isValid(true);
+    assert bottomProgram.isValid(Litmus.THROW);
+    assert topProgram.isValid(Litmus.THROW);
     final RexProgramBuilder progBuilder =
         RexProgramBuilder.forProgram(bottomProgram, rexBuilder, false);
 
@@ -745,7 +746,7 @@ public class RexProgramBuilder {
       progBuilder.addProject(pair.left, pair.right);
     }
     RexProgram mergedProg = progBuilder.getProgram(normalize);
-    assert mergedProg.isValid(true);
+    assert mergedProg.isValid(Litmus.THROW);
     assert mergedProg.getOutputRowType() == topProgram.getOutputRowType();
     return mergedProg;
   }
@@ -906,7 +907,7 @@ public class RexProgramBuilder {
         assert input.getType().isStruct()
             || RelOptUtil.eq("type1", input.getType(),
                 "type2", inputRowType.getFieldList().get(index).getType(),
-                true);
+                Litmus.THROW);
       }
 
       // Return a reference to the N'th expression, which should be
@@ -927,7 +928,7 @@ public class RexProgramBuilder {
             exprList.get(index).getType(),
             "ref type",
             local.getType(),
-            true);
+            Litmus.THROW);
       }
 
       // Resolve the expression to an input.
@@ -994,7 +995,7 @@ public class RexProgramBuilder {
           local.getType(),
           "type2",
           input.getType(),
-          true);
+          Litmus.THROW);
       return local;
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index d801d1b..8e8a79c 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -33,6 +33,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
@@ -476,34 +477,35 @@ public class RexUtil {
   /**
    * Returns whether an array of expressions has any common sub-expressions.
    */
-  public static boolean containCommonExprs(List<RexNode> exprs, boolean fail) {
+  public static boolean containNoCommonExprs(List<RexNode> exprs,
+      Litmus litmus) {
     final ExpressionNormalizer visitor = new ExpressionNormalizer(false);
     for (RexNode expr : exprs) {
       try {
         expr.accept(visitor);
       } catch (ExpressionNormalizer.SubExprExistsException e) {
         Util.swallow(e, null);
-        assert !fail;
-        return true;
+        return litmus.fail(null);
       }
     }
-    return false;
+    return litmus.succeed();
   }
 
   /**
-   * Returns whether an array of expressions contains a forward reference.
+   * Returns whether an array of expressions contains no forward references.
    * That is, if expression #i contains a {@link RexInputRef} referencing
    * field i or greater.
    *
    * @param exprs        Array of expressions
    * @param inputRowType Input row type
-   * @param fail         Whether to assert if there is a forward reference
+   * @param litmus       What to do if an error is detected (there is a
+   *                     forward reference)
+   *
    * @return Whether there is a forward reference
    */
-  public static boolean containForwardRefs(
-      List<RexNode> exprs,
+  public static boolean containNoForwardRefs(List<RexNode> exprs,
       RelDataType inputRowType,
-      boolean fail) {
+      Litmus litmus) {
     final ForwardRefFinder visitor = new ForwardRefFinder(inputRowType);
     for (int i = 0; i < exprs.size(); i++) {
       RexNode expr = exprs.get(i);
@@ -512,21 +514,20 @@ public class RexUtil {
         expr.accept(visitor);
       } catch (ForwardRefFinder.IllegalForwardRefException e) {
         Util.swallow(e, null);
-        assert !fail : "illegal forward reference in " + expr;
-        return true;
+        return litmus.fail("illegal forward reference in " + expr);
       }
     }
-    return false;
+    return litmus.succeed();
   }
 
   /**
-   * Returns whether an array of exp contains aggregate function calls whose
-   * arguments are not {@link RexInputRef}.s
+   * Returns whether an array of exp contains no aggregate function calls whose
+   * arguments are not {@link RexInputRef}s.
    *
    * @param exprs Expressions
-   * @param fail  Whether to assert if there is such a function call
+   * @param litmus  Whether to assert if there is such a function call
    */
-  static boolean containNonTrivialAggs(List<RexNode> exprs, boolean fail) {
+  static boolean containNoNonTrivialAggs(List<RexNode> exprs, Litmus litmus) {
     for (RexNode expr : exprs) {
       if (expr instanceof RexCall) {
         RexCall rexCall = (RexCall) expr;
@@ -534,14 +535,13 @@ public class RexUtil {
           for (RexNode operand : rexCall.operands) {
             if (!(operand instanceof RexLocalRef)
                 && !(operand instanceof RexLiteral)) {
-              assert !fail : "contains non trivial agg: " + operand;
-              return true;
+              return litmus.fail("contains non trivial agg: " + operand);
             }
           }
         }
       }
     }
-    return false;
+    return litmus.succeed();
   }
 
   /**
@@ -623,28 +623,29 @@ public class RexUtil {
    *
    * @param exprs Array of expressions
    * @param type  Type
-   * @param fail  Whether to fail if there is a mismatch
+   * @param litmus What to do if an error is detected (there is a mismatch)
+   *
    * @return Whether every expression has the same type as the corresponding
    * member of the struct type
-   * @see RelOptUtil#eq(String, RelDataType, String, RelDataType, boolean)
+   *
+   * @see RelOptUtil#eq(String, RelDataType, String, RelDataType, org.apache.calcite.util.Litmus)
    */
   public static boolean compatibleTypes(
       List<RexNode> exprs,
       RelDataType type,
-      boolean fail) {
+      Litmus litmus) {
     final List<RelDataTypeField> fields = type.getFieldList();
     if (exprs.size() != fields.size()) {
-      assert !fail : "rowtype mismatches expressions";
-      return false;
+      return litmus.fail("rowtype mismatches expressions");
     }
     for (int i = 0; i < fields.size(); i++) {
       final RelDataType exprType = exprs.get(i).getType();
       final RelDataType fieldType = fields.get(i).getType();
-      if (!RelOptUtil.eq("type1", exprType, "type2", fieldType, fail)) {
-        return false;
+      if (!RelOptUtil.eq("type1", exprType, "type2", fieldType, litmus)) {
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**
@@ -665,39 +666,35 @@ public class RexUtil {
   public static boolean containIdentity(
       List<? extends RexNode> exprs,
       RelDataType rowType,
-      boolean fail) {
+      Litmus litmus) {
     final List<RelDataTypeField> fields = rowType.getFieldList();
     if (exprs.size() < fields.size()) {
-      assert !fail : "exprs/rowType length mismatch";
-      return false;
+      return litmus.fail("exprs/rowType length mismatch");
     }
     for (int i = 0; i < fields.size(); i++) {
       if (!(exprs.get(i) instanceof RexInputRef)) {
-        assert !fail : "expr[" + i + "] is not a RexInputRef";
-        return false;
+        return litmus.fail("expr[" + i + "] is not a RexInputRef");
       }
       RexInputRef inputRef = (RexInputRef) exprs.get(i);
       if (inputRef.getIndex() != i) {
-        assert !fail : "expr[" + i + "] has ordinal "
-            + inputRef.getIndex();
-        return false;
+        return litmus.fail("expr[" + i + "] has ordinal "
+            + inputRef.getIndex());
       }
       if (!RelOptUtil.eq("type1",
           exprs.get(i).getType(),
           "type2",
-          fields.get(i).getType(),
-          fail)) {
-        return false;
+          fields.get(i).getType(), litmus)) {
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   /** Returns whether a list of expressions projects the incoming fields. */
   public static boolean isIdentity(List<? extends RexNode> exps,
       RelDataType inputRowType) {
     return inputRowType.getFieldCount() == exps.size()
-        && containIdentity(exps, inputRowType, false);
+        && containIdentity(exps, inputRowType, Litmus.IGNORE);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
index 58b0d52..8d5f380 100644
--- a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
+++ b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
@@ -30,7 +30,16 @@ import com.google.common.collect.ImmutableList;
  *
  * <p>A user-defined schema does not need to implement this interface, but by
  * the time a schema is passed to a method in a user-defined schema or
- * user-defined table, it will have been wrapped in this interface.</p>
+ * user-defined table, it will have been wrapped in this interface.
+ *
+ * <p>SchemaPlus is intended to be used by users but not instantiated by them.
+ * Users should only use the SchemaPlus they are given by the system.
+ * The purpose of SchemaPlus is to expose to user code, in a read only manner,
+ * some of the extra information about schemas that Calcite builds up when a
+ * schema is registered. It appears in several SPI calls as context; for example
+ * {@link SchemaFactory#create(SchemaPlus, String, java.util.Map)} contains a
+ * parent schema that might be a wrapped instance of a user-defined
+ * {@link Schema}, or indeed might not.
  */
 public interface SchemaPlus extends Schema {
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
index c5f9f66..34943c2 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.math.BigDecimal;
@@ -208,7 +209,7 @@ public class SqlBinaryOperator extends SqlOperator {
     return super.getMonotonicity(call);
   }
 
-  @Override public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, Litmus litmus) {
     if (count != 2) {
       // Special exception for AND and OR.
       if ((this == SqlStdOperatorTable.AND
@@ -216,10 +217,9 @@ public class SqlBinaryOperator extends SqlOperator {
           && count > 2) {
         return true;
       }
-      assert !fail : "wrong operand count " + count + " for " + this;
-      return false;
+      return litmus.fail("wrong operand count " + count + " for " + this);
     }
-    return true;
+    return litmus.succeed();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCall.java b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
index 3461b2d..59066b3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
@@ -24,6 +24,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -134,23 +135,21 @@ public abstract class SqlCall extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (node == this) {
       return true;
     }
     if (!(node instanceof SqlCall)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlCall that = (SqlCall) node;
 
     // Compare operators by name, not identity, because they may not
     // have been resolved yet.
     if (!this.getOperator().getName().equals(that.getOperator().getName())) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return equalDeep(this.getOperandList(), that.getOperandList(), fail);
+    return equalDeep(this.getOperandList(), that.getOperandList(), litmus);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
index 62abaa6..8ce7923 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Objects;
@@ -234,39 +235,33 @@ public class SqlDataTypeSpec extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlDataTypeSpec)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlDataTypeSpec that = (SqlDataTypeSpec) node;
     if (!SqlNode.equalDeep(
         this.collectionsTypeName,
-        that.collectionsTypeName,
-        fail)) {
-      return false;
+        that.collectionsTypeName, litmus)) {
+      return litmus.fail(null);
     }
-    if (!this.typeName.equalsDeep(that.typeName, fail)) {
-      return false;
+    if (!this.typeName.equalsDeep(that.typeName, litmus)) {
+      return litmus.fail(null);
     }
     if (this.precision != that.precision) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     if (this.scale != that.scale) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     if (!Objects.equal(this.timeZone, that.timeZone)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     if (!com.google.common.base.Objects.equal(this.charSetName,
         that.charSetName)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java b/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
index 552852c..8fc771c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
@@ -21,6 +21,7 @@ import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 
 /**
  * A <code>SqlDynamicParam</code> represents a dynamic parameter marker in an
@@ -76,17 +77,15 @@ public class SqlDynamicParam extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlDynamicParam)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlDynamicParam that = (SqlDynamicParam) node;
     if (this.index != that.index) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return true;
+    return litmus.succeed();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
index 9d0d633..bd0c5d5 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
@@ -22,6 +22,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlQualified;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -286,23 +287,20 @@ public class SqlIdentifier extends SqlNode {
     validator.validateIdentifier(this, scope);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlIdentifier)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlIdentifier that = (SqlIdentifier) node;
     if (this.names.size() != that.names.size()) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     for (int i = 0; i < names.size(); i++) {
       if (!this.names.get(i).equals(that.names.get(i))) {
-        assert !fail : this + "!=" + node;
-        return false;
+        return litmus.fail(this + "!=" + node);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   public <R> R accept(SqlVisitor<R> visitor) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
index 089cbf6..aa9729d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
@@ -142,14 +143,13 @@ public class SqlIntervalQualifier extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     final String thisString = this.toString();
     final String thatString = node.toString();
     if (!thisString.equals(thatString)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public int getStartPrecision(RelDataTypeSystem typeSystem) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index 3295fb5..32714c6 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -31,6 +31,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.BitString;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Util;
 
@@ -403,17 +404,15 @@ public class SqlLiteral extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlLiteral)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlLiteral that = (SqlLiteral) node;
     if (!this.equals(that)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public SqlMonotonicity getMonotonicity(SqlValidatorScope scope) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index 6531114..b31cf7b 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -24,6 +24,7 @@ import org.apache.calcite.sql.validate.SqlMoniker;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.util.Collection;
@@ -254,27 +255,32 @@ public abstract class SqlNode implements Cloneable {
    * (2 + 3), because the '+' operator is left-associative</li>
    * </ul>
    */
-  public abstract boolean equalsDeep(SqlNode node, boolean fail);
+  public abstract boolean equalsDeep(SqlNode node, Litmus litmus);
+
+  @Deprecated // to be removed before 2.0
+  public final boolean equalsDeep(SqlNode node, boolean fail) {
+    return equalsDeep(node, fail ? Litmus.THROW : Litmus.IGNORE);
+  }
 
   /**
    * Returns whether two nodes are equal (using
-   * {@link #equalsDeep(SqlNode, boolean)}) or are both null.
+   * {@link #equalsDeep(SqlNode, Litmus)}) or are both null.
    *
    * @param node1 First expression
    * @param node2 Second expression
-   * @param fail  Whether to throw {@link AssertionError} if expressions are
-   *              not equal
+   * @param litmus What to do if an error is detected (expressions are
+   *              not equal)
    */
   public static boolean equalDeep(
       SqlNode node1,
       SqlNode node2,
-      boolean fail) {
+      Litmus litmus) {
     if (node1 == null) {
       return node2 == null;
     } else if (node2 == null) {
       return false;
     } else {
-      return node1.equalsDeep(node2, fail);
+      return node1.equalsDeep(node2, litmus);
     }
   }
 
@@ -294,17 +300,16 @@ public abstract class SqlNode implements Cloneable {
 
   /** Returns whether two lists of operands are equal. */
   public static boolean equalDeep(List<SqlNode> operands0,
-      List<SqlNode> operands1, boolean fail) {
+      List<SqlNode> operands1, Litmus litmus) {
     if (operands0.size() != operands1.size()) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
     for (int i = 0; i < operands0.size(); i++) {
-      if (!SqlNode.equalDeep(operands0.get(i), operands1.get(i), fail)) {
-        return false;
+      if (!SqlNode.equalDeep(operands0.get(i), operands1.get(i), litmus)) {
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
index 4d77dac..392d967 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
@@ -21,6 +21,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -55,7 +56,7 @@ public class SqlNodeList extends SqlNode implements Iterable<SqlNode> {
    */
   public SqlNodeList(SqlParserPos pos) {
     super(pos);
-    list = new ArrayList<SqlNode>();
+    list = new ArrayList<>();
   }
 
   /**
@@ -66,7 +67,7 @@ public class SqlNodeList extends SqlNode implements Iterable<SqlNode> {
       Collection<? extends SqlNode> collection,
       SqlParserPos pos) {
     super(pos);
-    list = new ArrayList<SqlNode>(collection);
+    list = new ArrayList<>(collection);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -152,24 +153,22 @@ public class SqlNodeList extends SqlNode implements Iterable<SqlNode> {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlNodeList)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlNodeList that = (SqlNodeList) node;
     if (this.size() != that.size()) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     for (int i = 0; i < list.size(); i++) {
       SqlNode thisChild = list.get(i);
       final SqlNode thatChild = that.list.get(i);
-      if (!thisChild.equalsDeep(thatChild, fail)) {
-        return false;
+      if (!thisChild.equalsDeep(thatChild, litmus)) {
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   public SqlNode[] toArray() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
index df165ca..bef8b59 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.util.Arrays;
@@ -590,7 +591,7 @@ public abstract class SqlOperator {
    * (some examples are CAST and AND), and this method throws internal errors,
    * not user errors.</p>
    */
-  public boolean validRexOperands(int count, boolean fail) {
+  public boolean validRexOperands(int count, Litmus litmus) {
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
index 9b36a08..5fef8ae 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
@@ -22,6 +22,7 @@ import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 /**
@@ -88,12 +89,11 @@ public class SqlPostfixOperator extends SqlOperator {
     return type;
   }
 
-  @Override public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, Litmus litmus) {
     if (count != 1) {
-      assert !fail : "wrong operand count " + count + " for " + this;
-      return false;
+      return litmus.fail("wrong operand count " + count + " for " + this);
     }
-    return true;
+    return litmus.succeed();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
index 8923645..8976a56 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 /**
@@ -97,12 +98,11 @@ public class SqlPrefixOperator extends SqlOperator {
     return super.getMonotonicity(call);
   }
 
-  @Override public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, Litmus litmus) {
     if (count != 1) {
-      assert !fail : "wrong operand count " + count + " for " + this;
-      return false;
+      return litmus.fail("wrong operand count " + count + " for " + this);
     }
-    return true;
+    return litmus.succeed();
   }
 }
 


[36/50] [abbrv] calcite git commit: [CALCITE-996] Simplify predicate when we create a Filter operator

Posted by jh...@apache.org.
[CALCITE-996] Simplify predicate when we create a Filter operator

Close apache/calcite#171


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

Branch: refs/heads/branch-release
Commit: a67b4a976bef6c104212732c6a8f8ce364c372ba
Parents: ebcba3b
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Mon Jan 11 21:12:30 2016 +0100
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 12 10:21:49 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/tools/RelBuilder.java    |  6 +++---
 .../org/apache/calcite/test/RelBuilderTest.java | 22 ++++++++++++++++++++
 .../calcite/test/SqlToRelConverterTest.xml      | 11 ++++------
 3 files changed, 29 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/a67b4a97/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 916f0dd..00ce6d1 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -692,9 +692,9 @@ public class RelBuilder {
    * and optimized in a similar way to the {@link #and} method.
    * If the result is TRUE no filter is created. */
   public RelBuilder filter(Iterable<? extends RexNode> predicates) {
-    final RexNode x = RexUtil.composeConjunction(cluster.getRexBuilder(),
-        predicates, true);
-    if (x != null) {
+    final RexNode x = RexUtil.simplify(cluster.getRexBuilder(),
+            RexUtil.composeConjunction(cluster.getRexBuilder(), predicates, false));
+    if (!x.isAlwaysTrue()) {
       final Frame frame = Stacks.pop(stack);
       final RelNode filter = filterFactory.createFilter(frame.rel, x);
       Stacks.push(stack, new Frame(filter, frame.right));

http://git-wip-us.apache.org/repos/asf/calcite/blob/a67b4a97/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 bb53728..0b232f4 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -202,6 +202,28 @@ public class RelBuilderTest {
             + "  LogicalTableScan(table=[[scott, EMP]])\n"));
   }
 
+  @Test public void testScanFilterOr2() {
+    // Equivalent SQL:
+    //   SELECT *
+    //   FROM emp
+    //   WHERE deptno = 20 OR deptno = 20
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("EMP")
+            .filter(
+                builder.call(SqlStdOperatorTable.OR,
+                    builder.call(SqlStdOperatorTable.GREATER_THAN,
+                        builder.field("DEPTNO"),
+                        builder.literal(20)),
+                    builder.call(SqlStdOperatorTable.GREATER_THAN,
+                        builder.field("DEPTNO"),
+                        builder.literal(20))))
+            .build();
+    assertThat(str(root),
+        is("LogicalFilter(condition=[>($7, 20)])\n"
+            + "  LogicalTableScan(table=[[scott, EMP]])\n"));
+  }
+
   @Test public void testBadFieldName() {
     final RelBuilder builder = RelBuilder.create(config().build());
     try {

http://git-wip-us.apache.org/repos/asf/calcite/blob/a67b4a97/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 764dde2..a02eb60 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -511,8 +511,7 @@ LogicalProject(EXPR$0=[1])
     <TestCase name="testIntervalLiteralYearToMonth">
         <Resource name="sql">
             <![CDATA[select cast(empno as Integer) * (INTERVAL '1-1' YEAR TO MONTH)
-from emp
-]]>
+from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -524,8 +523,7 @@ LogicalProject(EXPR$0=[*(CAST($0):INTEGER NOT NULL, 13)])
     <TestCase name="testIntervalLiteralHourToMinute">
         <Resource name="sql">
             <![CDATA[select cast(empno as Integer) * (INTERVAL '1:1' HOUR TO MINUTE)
-from emp
-]]>
+from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -2394,8 +2392,7 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[RANK() OVER (ORDER BY $1 RANGE BETWEEN UNBOU
         <Resource name="sql">
             <![CDATA[select avg(deptno) over ()
 from emp
-group by deptno
-]]>
+group by deptno]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -2562,7 +2559,7 @@ and not exists (select * from emp e2 where e1.empno = e2.empno)]]>
             <![CDATA[
 LogicalProject(EMPNO=[$0])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10], $f0=[$12])
-    LogicalFilter(condition=[NOT(IS NOT NULL($12))])
+    LogicalFilter(condition=[IS NULL($12)])
       LogicalJoin(condition=[=($0, $11)], joinType=[left])
         LogicalJoin(condition=[=($7, $9)], joinType=[inner])
           LogicalFilter(condition=[<($7, 10)])


[33/50] [abbrv] calcite git commit: [CALCITE-794] Detect cycles when computing statistics

Posted by jh...@apache.org.
[CALCITE-794] Detect cycles when computing statistics

Make RelMetadataQuery methods non-static. Each active call is
registered in the RelMetadataQuery instance; it throws
CyclicMetadataException when it detects a cycle, and the caller can
catch it to return a sensible default value for the particular kind of
metadata.

Change signature of FlatLists.of(Object...) to
FlatLists.copyOf(Comparable...).

Temporarily disable all failing tests.

Add metadata for EnumerableLimit.


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

Branch: refs/heads/branch-release
Commit: cabdcf44e4aec4d4ceea7f97c8c6fd9e9dbd36b1
Parents: 2cef859
Author: Julian Hyde <jh...@apache.org>
Authored: Sat Jul 11 14:56:42 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:25 2016 -0800

----------------------------------------------------------------------
 .../adapter/enumerable/EnumerableCalc.java      |   8 +-
 .../adapter/enumerable/EnumerableFilter.java    |   6 +-
 .../enumerable/EnumerableInterpreter.java       |   6 +-
 .../adapter/enumerable/EnumerableJoin.java      |   9 +-
 .../adapter/enumerable/EnumerableLimit.java     |  25 +-
 .../adapter/enumerable/EnumerableMergeJoin.java |  12 +-
 .../adapter/enumerable/EnumerableProject.java   |   4 +-
 .../adapter/enumerable/EnumerableSemiJoin.java  |   9 +-
 .../adapter/enumerable/EnumerableThetaJoin.java |   9 +-
 .../adapter/enumerable/EnumerableValues.java    |   4 +-
 .../adapter/enumerable/EnumerableWindow.java    |   5 +-
 .../adapter/enumerable/JavaRowFormat.java       |   2 +-
 .../adapter/enumerable/PhysTypeImpl.java        |   2 +-
 .../apache/calcite/adapter/jdbc/JdbcRules.java  |  33 ++-
 .../adapter/jdbc/JdbcToEnumerableConverter.java |   6 +-
 .../apache/calcite/interpreter/Bindables.java   |  14 +-
 .../calcite/plan/AbstractRelOptPlanner.java     |  30 +--
 .../apache/calcite/plan/ConventionTraitDef.java |   4 +-
 .../org/apache/calcite/plan/RelOptCost.java     |   2 +-
 .../org/apache/calcite/plan/RelOptPlanner.java  |  10 +-
 .../org/apache/calcite/plan/hep/HepPlanner.java |  57 +++--
 .../plan/hep/HepRelMetadataProvider.java        |  22 +-
 .../apache/calcite/plan/hep/HepRelVertex.java   |   7 +-
 .../calcite/plan/volcano/AbstractConverter.java |   3 +-
 .../org/apache/calcite/plan/volcano/RelSet.java |   7 +-
 .../apache/calcite/plan/volcano/RelSubset.java  |  46 ++--
 .../apache/calcite/plan/volcano/RuleQueue.java  |  14 +-
 .../calcite/plan/volcano/VolcanoPlanner.java    |  23 +-
 .../volcano/VolcanoRelMetadataProvider.java     |  31 +--
 .../org/apache/calcite/prepare/PlannerImpl.java |   5 +
 .../org/apache/calcite/rel/AbstractRelNode.java |  28 +-
 .../java/org/apache/calcite/rel/RelNode.java    |  24 +-
 .../java/org/apache/calcite/rel/SingleRel.java  |   4 +-
 .../calcite/rel/convert/ConverterImpl.java      |   6 +-
 .../org/apache/calcite/rel/core/Aggregate.java  |   9 +-
 .../java/org/apache/calcite/rel/core/Calc.java  |  18 +-
 .../org/apache/calcite/rel/core/Correlate.java  |  13 +-
 .../org/apache/calcite/rel/core/Exchange.java   |   5 +-
 .../org/apache/calcite/rel/core/Filter.java     |  35 +--
 .../org/apache/calcite/rel/core/Intersect.java  |   5 +-
 .../java/org/apache/calcite/rel/core/Join.java  |  14 +-
 .../java/org/apache/calcite/rel/core/Minus.java |   5 +-
 .../org/apache/calcite/rel/core/Project.java    |   5 +-
 .../org/apache/calcite/rel/core/SemiJoin.java   |  10 +-
 .../java/org/apache/calcite/rel/core/Sort.java  |   5 +-
 .../calcite/rel/core/TableFunctionScan.java     |  10 +-
 .../apache/calcite/rel/core/TableModify.java    |   8 +-
 .../org/apache/calcite/rel/core/TableScan.java  |   6 +-
 .../java/org/apache/calcite/rel/core/Union.java |  21 +-
 .../org/apache/calcite/rel/core/Values.java     |  13 +-
 .../org/apache/calcite/rel/core/Window.java     |   5 +-
 .../calcite/rel/externalize/RelWriterImpl.java  |  15 +-
 .../apache/calcite/rel/logical/LogicalCalc.java |   6 +-
 .../calcite/rel/logical/LogicalFilter.java      |   6 +-
 .../calcite/rel/logical/LogicalProject.java     |   4 +-
 .../rel/logical/LogicalTableFunctionScan.java   |   3 +-
 .../calcite/rel/logical/LogicalValues.java      |   4 +-
 .../calcite/rel/metadata/BuiltInMetadata.java   |  14 +-
 .../metadata/CachingRelMetadataProvider.java    |  53 ++--
 .../metadata/ChainedRelMetadataProvider.java    |  44 ++--
 .../rel/metadata/CyclicMetadataException.java   |  37 +++
 .../apache/calcite/rel/metadata/Metadata.java   |   2 +-
 .../calcite/rel/metadata/MetadataFactory.java   |  13 +-
 .../rel/metadata/MetadataFactoryImpl.java       |  31 +--
 .../metadata/ReflectiveRelMetadataProvider.java | 100 +++++---
 .../calcite/rel/metadata/RelMdCollation.java    | 102 ++++----
 .../rel/metadata/RelMdColumnOrigins.java        | 110 +++-----
 .../rel/metadata/RelMdColumnUniqueness.java     | 255 +++++++++----------
 .../rel/metadata/RelMdDistinctRowCount.java     | 171 ++++++-------
 .../calcite/rel/metadata/RelMdDistribution.java |  47 ++--
 .../rel/metadata/RelMdExplainVisibility.java    |  10 +-
 .../calcite/rel/metadata/RelMdMaxRowCount.java  |  69 +++--
 .../calcite/rel/metadata/RelMdMemory.java       |  24 +-
 .../calcite/rel/metadata/RelMdParallelism.java  |  14 +-
 .../metadata/RelMdPercentageOriginalRows.java   |  46 ++--
 .../rel/metadata/RelMdPopulationSize.java       |  73 +++---
 .../calcite/rel/metadata/RelMdPredicates.java   | 136 +++++-----
 .../calcite/rel/metadata/RelMdRowCount.java     | 129 +++++++---
 .../calcite/rel/metadata/RelMdSelectivity.java  |  60 ++---
 .../apache/calcite/rel/metadata/RelMdSize.java  |  66 +++--
 .../calcite/rel/metadata/RelMdUniqueKeys.java   |  60 ++---
 .../apache/calcite/rel/metadata/RelMdUtil.java  | 255 +++++++++----------
 .../rel/metadata/RelMetadataProvider.java       |   8 +-
 .../calcite/rel/metadata/RelMetadataQuery.java  | 159 ++++++------
 .../calcite/rel/metadata/UnboundMetadata.java   |  31 +++
 .../rel/rules/AggregateFilterTransposeRule.java |   4 +-
 .../rel/rules/AggregateJoinTransposeRule.java   |  12 +-
 .../AggregateProjectPullUpConstantsRule.java    |   3 +-
 .../calcite/rel/rules/AggregateRemoveRule.java  |   9 +-
 .../rel/rules/AggregateStarTableRule.java       |   4 +-
 .../rel/rules/AggregateUnionTransposeRule.java  |   5 +-
 .../rules/JoinPushTransitivePredicatesRule.java |   3 +-
 .../apache/calcite/rel/rules/LoptMultiJoin.java |  19 +-
 .../calcite/rel/rules/LoptOptimizeJoinRule.java |  63 ++---
 .../rel/rules/LoptSemiJoinOptimizer.java        |  39 ++-
 .../rel/rules/MultiJoinOptimizeBushyRule.java   |   3 +-
 .../rel/rules/ReduceExpressionsRule.java        |  13 +-
 .../rel/rules/SortJoinTransposeRule.java        |  10 +-
 .../rel/rules/SortUnionTransposeRule.java       |   6 +-
 .../calcite/rel/rules/SubQueryRemoveRule.java   |   3 +-
 .../org/apache/calcite/runtime/FlatLists.java   |  75 +++++-
 .../apache/calcite/schema/impl/StarTable.java   |   4 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java |  14 +-
 .../apache/calcite/sql2rel/RelFieldTrimmer.java |  10 +-
 .../main/java/org/apache/calcite/util/Bug.java  |   6 +-
 .../org/apache/calcite/util/BuiltInMethod.java  |   6 +-
 .../org/apache/calcite/util/NumberUtil.java     |  11 +
 .../plan/volcano/TraitPropagationTest.java      |  18 +-
 .../plan/volcano/VolcanoPlannerTest.java        |  13 +-
 .../plan/volcano/VolcanoPlannerTraitTest.java   |  22 +-
 .../apache/calcite/test/JdbcAdapterTest.java    |  28 +-
 .../java/org/apache/calcite/test/JdbcTest.java  |   8 +-
 .../apache/calcite/test/RelMetadataTest.java    | 159 +++++++-----
 .../org/apache/calcite/tools/PlannerTest.java   |   9 +-
 .../java/org/apache/calcite/util/UtilTest.java  |  10 +
 core/src/test/resources/sql/agg.iq              |   2 +-
 .../calcite/adapter/mongodb/MongoFilter.java    |   6 +-
 .../calcite/adapter/mongodb/MongoProject.java   |   6 +-
 .../calcite/adapter/mongodb/MongoSort.java      |   6 +-
 .../calcite/adapter/mongodb/MongoTableScan.java |   6 +-
 .../mongodb/MongoToEnumerableConverter.java     |   6 +-
 .../spark/EnumerableToSparkConverter.java       |   6 +-
 .../adapter/spark/JdbcToSparkConverter.java     |   6 +-
 .../calcite/adapter/spark/SparkRules.java       |  16 +-
 .../spark/SparkToEnumerableConverter.java       |   6 +-
 125 files changed, 1912 insertions(+), 1564 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
index ce1f642..205bd05 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalc.java
@@ -37,6 +37,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Calc;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Pair;
@@ -86,18 +87,19 @@ public class EnumerableCalc extends Calc implements EnumerableRel {
   public static EnumerableCalc create(final RelNode input,
       final RexProgram program) {
     final RelOptCluster cluster = input.getCluster();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet = cluster.traitSet()
         .replace(EnumerableConvention.INSTANCE)
         .replaceIfs(RelCollationTraitDef.INSTANCE,
             new Supplier<List<RelCollation>>() {
               public List<RelCollation> get() {
-                return RelMdCollation.calc(input, program);
+                return RelMdCollation.calc(mq, input, program);
               }
             })
         .replaceIf(RelDistributionTraitDef.INSTANCE,
             new Supplier<RelDistribution>() {
               public RelDistribution get() {
-                return RelMdDistribution.calc(input, program);
+                return RelMdDistribution.calc(mq, input, program);
               }
             });
     return new EnumerableCalc(cluster, traitSet, input, program);
@@ -207,7 +209,7 @@ public class EnumerableCalc extends Calc implements EnumerableRel {
         Expressions.new_(
             enumeratorType,
             NO_EXPRS,
-            Expressions.<MemberDeclaration>list(
+            Expressions.list(
                 Expressions.fieldDecl(
                     Modifier.PUBLIC
                     | Modifier.FINAL,

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java
index 7dccf91..894ff16 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilter.java
@@ -26,6 +26,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 
 import com.google.common.base.Supplier;
@@ -53,19 +54,20 @@ public class EnumerableFilter
   public static EnumerableFilter create(final RelNode input,
       RexNode condition) {
     final RelOptCluster cluster = input.getCluster();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet =
         cluster.traitSetOf(EnumerableConvention.INSTANCE)
             .replaceIfs(
                 RelCollationTraitDef.INSTANCE,
                 new Supplier<List<RelCollation>>() {
                   public List<RelCollation> get() {
-                    return RelMdCollation.filter(input);
+                    return RelMdCollation.filter(mq, input);
                   }
                 })
             .replaceIf(RelDistributionTraitDef.INSTANCE,
                 new Supplier<RelDistribution>() {
                   public RelDistribution get() {
-                    return RelMdDistribution.filter(input);
+                    return RelMdDistribution.filter(mq, input);
                   }
                 });
     return new EnumerableFilter(cluster, traitSet, input, condition);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java
index cfa241f..1c53483 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreter.java
@@ -27,6 +27,7 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.BuiltInMethod;
 
 import java.util.List;
@@ -72,8 +73,9 @@ public class EnumerableInterpreter extends SingleRel
         factor);
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(factor);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(factor);
   }
 
   @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
index 0b86771..cfd00a4 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
@@ -125,8 +125,9 @@ public class EnumerableJoin extends EquiJoin implements EnumerableRel {
     }
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double rowCount = RelMetadataQuery.getRowCount(this);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double rowCount = mq.getRowCount(this);
 
     // Joins can be flipped, and for many algorithms, both versions are viable
     // and have the same cost. To make the results stable between versions of
@@ -143,8 +144,8 @@ public class EnumerableJoin extends EquiJoin implements EnumerableRel {
 
     // Cheaper if the smaller number of rows is coming from the LHS.
     // Model this by adding L log L to the cost.
-    final double rightRowCount = right.getRows();
-    final double leftRowCount = left.getRows();
+    final double rightRowCount = right.estimateRowCount(mq);
+    final double leftRowCount = left.estimateRowCount(mq);
     if (Double.isInfinite(leftRowCount)) {
       rowCount = leftRowCount;
     } else {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java
index cea8755..827944f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.BuiltInMethod;
@@ -40,8 +41,8 @@ import java.util.List;
 
 /** Relational expression that applies a limit and/or offset to its input. */
 public class EnumerableLimit extends SingleRel implements EnumerableRel {
-  private final RexNode offset;
-  private final RexNode fetch;
+  public final RexNode offset;
+  public final RexNode fetch;
 
   /** Creates an EnumerableLimit.
    *
@@ -63,19 +64,20 @@ public class EnumerableLimit extends SingleRel implements EnumerableRel {
   public static EnumerableLimit create(final RelNode input, RexNode offset,
       RexNode fetch) {
     final RelOptCluster cluster = input.getCluster();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet =
         cluster.traitSetOf(EnumerableConvention.INSTANCE)
             .replaceIfs(
                 RelCollationTraitDef.INSTANCE,
                 new Supplier<List<RelCollation>>() {
                   public List<RelCollation> get() {
-                    return RelMdCollation.limit(input);
+                    return RelMdCollation.limit(mq, input);
                   }
                 })
             .replaceIf(RelDistributionTraitDef.INSTANCE,
                 new Supplier<RelDistribution>() {
                   public RelDistribution get() {
-                    return RelMdDistribution.limit(input);
+                    return RelMdDistribution.limit(mq, input);
                   }
                 });
     return new EnumerableLimit(cluster, traitSet, input, offset, fetch);
@@ -98,21 +100,6 @@ public class EnumerableLimit extends SingleRel implements EnumerableRel {
         .itemIf("fetch", fetch, fetch != null);
   }
 
-  @Override public double getRows() {
-    double rowCount = super.getRows();
-    final int offset =
-        this.offset == null ? 0 : RexLiteral.intValue(this.offset);
-    rowCount = Math.max(rowCount - offset, 0D);
-
-    if (this.fetch != null) {
-      final int limit = RexLiteral.intValue(this.fetch);
-      if (limit < rowCount) {
-        return (double) limit;
-      }
-    }
-    return rowCount;
-  }
-
   public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
     final BlockBuilder builder = new BlockBuilder();
     final EnumerableRel child = (EnumerableRel) getInput();

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java
index 18419e3..fd492cb 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoin.java
@@ -83,8 +83,9 @@ public class EnumerableMergeJoin extends EquiJoin implements EnumerableRel {
     final RelOptCluster cluster = right.getCluster();
     RelTraitSet traitSet = cluster.traitSet();
     if (traitSet.isEnabled(RelCollationTraitDef.INSTANCE)) {
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
       final List<RelCollation> collations =
-          RelMdCollation.mergeJoin(left, right, leftKeys, rightKeys);
+          RelMdCollation.mergeJoin(mq, left, right, leftKeys, rightKeys);
       traitSet = traitSet.replace(collations);
     }
     return new EnumerableMergeJoin(cluster, traitSet, left, right, condition,
@@ -107,13 +108,14 @@ public class EnumerableMergeJoin extends EquiJoin implements EnumerableRel {
     }
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // We assume that the inputs are sorted. The price of sorting them has
     // already been paid. The cost of the join is therefore proportional to the
     // input and output size.
-    final double rightRowCount = right.getRows();
-    final double leftRowCount = left.getRows();
-    final double rowCount = RelMetadataQuery.getRowCount(this);
+    final double rightRowCount = right.estimateRowCount(mq);
+    final double leftRowCount = left.estimateRowCount(mq);
+    final double rowCount = mq.getRowCount(this);
     final double d = leftRowCount + rightRowCount + rowCount;
     return planner.getCostFactory().makeCost(d, 0, 0);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java
index e6852f7..fa2b48b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProject.java
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.metadata.RelMdCollation;
+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.Util;
@@ -68,12 +69,13 @@ public class EnumerableProject extends Project implements EnumerableRel {
   public static EnumerableProject create(final RelNode input,
       final List<? extends RexNode> projects, RelDataType rowType) {
     final RelOptCluster cluster = input.getCluster();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet =
         cluster.traitSet().replace(EnumerableConvention.INSTANCE)
             .replaceIfs(RelCollationTraitDef.INSTANCE,
                 new Supplier<List<RelCollation>>() {
                   public List<RelCollation> get() {
-                    return RelMdCollation.project(input, projects);
+                    return RelMdCollation.project(mq, input, projects);
                   }
                 });
     return new EnumerableProject(cluster, traitSet, input, projects, rowType);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java
index 569e79e..0162c51 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSemiJoin.java
@@ -83,12 +83,13 @@ public class EnumerableSemiJoin extends SemiJoin implements EnumerableRel {
     }
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double rowCount = RelMetadataQuery.getRowCount(this);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double rowCount = mq.getRowCount(this);
 
     // Right-hand input is the "build", and hopefully small, input.
-    final double rightRowCount = right.getRows();
-    final double leftRowCount = left.getRows();
+    final double rightRowCount = right.estimateRowCount(mq);
+    final double leftRowCount = left.estimateRowCount(mq);
     if (Double.isInfinite(leftRowCount)) {
       rowCount = leftRowCount;
     } else {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java
index e28ddfc..dc400ad 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableThetaJoin.java
@@ -73,8 +73,9 @@ public class EnumerableThetaJoin extends Join implements EnumerableRel {
     }
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double rowCount = RelMetadataQuery.getRowCount(this);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double rowCount = mq.getRowCount(this);
 
     // Joins can be flipped, and for many algorithms, both versions are viable
     // and have the same cost. To make the results stable between versions of
@@ -89,8 +90,8 @@ public class EnumerableThetaJoin extends Join implements EnumerableRel {
       }
     }
 
-    final double rightRowCount = right.getRows();
-    final double leftRowCount = left.getRows();
+    final double rightRowCount = right.estimateRowCount(mq);
+    final double leftRowCount = left.estimateRowCount(mq);
     if (Double.isInfinite(leftRowCount)) {
       rowCount = leftRowCount;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
index 13dd24d..6023c02 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexLiteral;
@@ -57,12 +58,13 @@ public class EnumerableValues extends Values implements EnumerableRel {
   public static EnumerableValues create(RelOptCluster cluster,
       final RelDataType rowType,
       final ImmutableList<ImmutableList<RexLiteral>> tuples) {
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelTraitSet traitSet =
         cluster.traitSetOf(EnumerableConvention.INSTANCE)
             .replaceIfs(RelCollationTraitDef.INSTANCE,
                 new Supplier<List<RelCollation>>() {
                   public List<RelCollation> get() {
-                    return RelMdCollation.values(rowType, tuples);
+                    return RelMdCollation.values(mq, rowType, tuples);
                   }
                 })
             .replaceIf(RelDistributionTraitDef.INSTANCE,

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 433c6cd..f8bd84f 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
@@ -38,6 +38,7 @@ import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Window;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexInputRef;
@@ -74,8 +75,8 @@ public class EnumerableWindow extends Window implements EnumerableRel {
         constants, rowType, groups);
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner)
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq)
         .multiplyBy(EnumerableConvention.COST_MULTIPLIER);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
index 9cb54ff..316af0d 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
@@ -148,7 +148,7 @@ public enum JavaRowFormat {
                 null,
                 BuiltInMethod.LIST_N.method,
                 Expressions.newArrayInit(
-                    Object.class,
+                    Comparable.class,
                     expressions)),
             List.class);
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
index 2c0c491..0f83889 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
@@ -599,7 +599,7 @@ public class PhysTypeImpl implements PhysType {
                 null,
                 BuiltInMethod.LIST_N.method,
                 Expressions.newArrayInit(
-                    Object.class,
+                    Comparable.class,
                     list)),
             v1);
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index ca80d29..db8bc89 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -56,6 +56,7 @@ import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalTableModify;
 import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rel2sql.SqlImplementor;
 import org.apache.calcite.rel.type.RelDataType;
@@ -247,16 +248,17 @@ public class JdbcRules {
       }
     }
 
-    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
       // We always "build" the
-      double rowCount = RelMetadataQuery.getRowCount(this);
+      double rowCount = mq.getRowCount(this);
 
       return planner.getCostFactory().makeCost(rowCount, 0, 0);
     }
 
-    @Override public double getRows() {
-      final double leftRowCount = left.getRows();
-      final double rightRowCount = right.getRows();
+    @Override public double estimateRowCount(RelMetadataQuery mq) {
+      final double leftRowCount = left.estimateRowCount(mq);
+      final double rightRowCount = right.estimateRowCount(mq);
       return Math.max(leftRowCount, rightRowCount);
     }
 
@@ -316,13 +318,14 @@ public class JdbcRules {
       return program.explainCalc(super.explainTerms(pw));
     }
 
-    public double getRows() {
-      return LogicalFilter.estimateFilteredRows(getInput(), program);
+    @Override public double estimateRowCount(RelMetadataQuery mq) {
+      return RelMdUtil.estimateFilteredRows(getInput(), program, mq);
     }
 
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      double dRows = RelMetadataQuery.getRowCount(this);
-      double dCpu = RelMetadataQuery.getRowCount(getInput())
+    public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      double dRows = mq.getRowCount(this);
+      double dCpu = mq.getRowCount(getInput())
           * program.getExprCount();
       double dIo = 0;
       return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
@@ -387,8 +390,9 @@ public class JdbcRules {
       return new JdbcProject(getCluster(), traitSet, input, projects, rowType);
     }
 
-    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      return super.computeSelfCost(planner)
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return super.computeSelfCost(planner, mq)
           .multiplyBy(JdbcConvention.COST_MULTIPLIER);
     }
 
@@ -604,8 +608,9 @@ public class JdbcRules {
       return new JdbcUnion(getCluster(), traitSet, inputs, all);
     }
 
-    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      return super.computeSelfCost(planner).multiplyBy(.1);
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return super.computeSelfCost(planner, mq).multiplyBy(.1);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
index 125d61b..732772d 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
@@ -36,6 +36,7 @@ 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.runtime.SqlFunctions;
@@ -71,8 +72,9 @@ public class JdbcToEnumerableConverter
         getCluster(), traitSet, sole(inputs));
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(.1);
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    return super.computeSelfCost(planner, mq).multiplyBy(.1);
   }
 
   public Result implement(EnumerableRelImplementor implementor, Prefer pref) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
index 9d4b250..aa44c6a 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
@@ -56,6 +56,7 @@ import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.logical.LogicalWindow;
 import org.apache.calcite.rel.metadata.RelMdCollation;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -217,8 +218,9 @@ public class Bindables {
           .itemIf("projects", projects, !projects.equals(identity()));
     }
 
-    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      return super.computeSelfCost(planner).multiplyBy(0.01d);
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return super.computeSelfCost(planner, mq).multiplyBy(0.01d);
     }
 
     public static boolean canHandle(RelOptTable table) {
@@ -267,12 +269,13 @@ public class Bindables {
     public static BindableFilter create(final RelNode input,
         RexNode condition) {
       final RelOptCluster cluster = input.getCluster();
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
       final RelTraitSet traitSet =
           cluster.traitSetOf(BindableConvention.INSTANCE)
               .replaceIfs(RelCollationTraitDef.INSTANCE,
                   new Supplier<List<RelCollation>>() {
                     public List<RelCollation> get() {
-                      return RelMdCollation.filter(input);
+                      return RelMdCollation.filter(mq, input);
                     }
                   });
       return new BindableFilter(cluster, traitSet, input, condition);
@@ -638,8 +641,9 @@ public class Bindables {
           constants, rowType, groups);
     }
 
-    @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-      return super.computeSelfCost(planner)
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return super.computeSelfCost(planner, mq)
           .multiplyBy(BindableConvention.COST_MULTIPLIER);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java b/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
index b25c28c..38c0e7e 100644
--- a/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
@@ -23,10 +23,10 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.CancelFlag;
 
 import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -53,8 +53,7 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
    * Maps rule description to rule, just to ensure that rules' descriptions
    * are unique.
    */
-  private final Map<String, RelOptRule> mapDescToRule =
-      new HashMap<String, RelOptRule>();
+  private final Map<String, RelOptRule> mapDescToRule = new HashMap<>();
 
   protected final RelOptCostFactory costFactory;
 
@@ -64,11 +63,9 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
 
   private CancelFlag cancelFlag;
 
-  @SuppressWarnings("unchecked")
-  private final Set<Class<? extends RelNode>> classes =
-      new HashSet<Class<? extends RelNode>>();
+  private final Set<Class<? extends RelNode>> classes = new HashSet<>();
 
-  private final Set<RelTrait> traits = new HashSet<RelTrait>();
+  private final Set<RelTrait> traits = new HashSet<>();
 
   /** External context. Never null. */
   protected final Context context;
@@ -111,7 +108,6 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
     return costFactory;
   }
 
-  // implement RelOptPlanner
   public void setCancelFlag(CancelFlag cancelFlag) {
     this.cancelFlag = cancelFlag;
   }
@@ -179,7 +175,6 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
     return mapDescToRule.get(description);
   }
 
-  // implement RelOptPlanner
   public void setRuleDescExclusionFilter(Pattern exclusionFilter) {
     ruleDescExclusionFilter = exclusionFilter;
   }
@@ -195,7 +190,6 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
         && ruleDescExclusionFilter.matcher(rule.toString()).matches();
   }
 
-  // implement RelOptPlanner
   public RelOptPlanner chooseDelegate() {
     return this;
   }
@@ -213,11 +207,9 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
     return null;
   }
 
-  // implement RelOptPlanner
   public void registerSchema(RelOptSchema schema) {
   }
 
-  // implement RelOptPlanner
   public long getRelMetadataTimestamp(RelNode rel) {
     return 0;
   }
@@ -246,12 +238,15 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
     return RelTraitSet.createEmpty();
   }
 
-  // implement RelOptPlanner
+  public RelOptCost getCost(RelNode rel, RelMetadataQuery mq) {
+    return mq.getCumulativeCost(rel);
+  }
+
   public RelOptCost getCost(RelNode rel) {
-    return RelMetadataQuery.getCumulativeCost(rel);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return getCost(rel, mq);
   }
 
-  // implement RelOptPlanner
   public void addListener(RelOptListener newListener) {
     if (listener == null) {
       listener = new MulticastRelOptListener();
@@ -259,20 +254,17 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
     listener.addListener(newListener);
   }
 
-  // implement RelOptPlanner
   public void registerMetadataProviders(List<RelMetadataProvider> list) {
   }
 
-  // implement RelOptPlanner
   public boolean addRelTraitDef(RelTraitDef relTraitDef) {
     return false;
   }
 
-  // implement RelOptPlanner
   public void clearRelTraitDefs() {}
 
   public List<RelTraitDef> getRelTraitDefs() {
-    return Collections.emptyList();
+    return ImmutableList.of();
   }
 
   public void setExecutor(Executor executor) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java b/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
index 9de4762..579e217 100644
--- a/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
+++ b/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
@@ -18,6 +18,7 @@ package org.apache.calcite.plan;
 
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.graph.DefaultDirectedGraph;
@@ -124,6 +125,7 @@ public class ConventionTraitDef extends RelTraitDef<Convention> {
       RelNode rel,
       Convention toConvention,
       boolean allowInfiniteCostConverters) {
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final ConversionData conversionData = getConversionData(planner);
 
     final Convention fromConvention = rel.getConvention();
@@ -139,7 +141,7 @@ public class ConventionTraitDef extends RelTraitDef<Convention> {
       RelNode converted = rel;
       Convention previous = null;
       for (Convention arc : conversionPath) {
-        if (planner.getCost(converted).isInfinite()
+        if (planner.getCost(converted, mq).isInfinite()
             && !allowInfiniteCostConverters) {
           continue loop;
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptCost.java b/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
index 1e19a72..bf9a55d 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
@@ -32,7 +32,7 @@ public interface RelOptCost {
   /**
    * @return number of rows processed; this should not be confused with the
    * row count produced by a relational expression
-   * ({@link org.apache.calcite.rel.RelNode#getRows})
+   * ({@link org.apache.calcite.rel.RelNode#estimateRowCount})
    */
   double getRows();
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java b/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
index 0e42cb7..9b5e22a 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
@@ -192,9 +192,17 @@ public interface RelOptPlanner {
    * Computes the cost of a RelNode. In most cases, this just dispatches to
    * {@link RelMetadataQuery#getCumulativeCost}.
    *
-   * @param rel expression of interest
+   * @param rel Relational expression of interest
+   * @param mq Metadata query
    * @return estimated cost
    */
+  RelOptCost getCost(RelNode rel, RelMetadataQuery mq);
+
+  /**
+   * @deprecated Use {@link #getCost(RelNode, RelMetadataQuery)}
+   * or, better, call {@link RelMetadataQuery#getCumulativeCost(RelNode)}.
+   */
+  @Deprecated // to be removed before 2.0
   RelOptCost getCost(RelNode rel);
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java b/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
index 30e28b0..5569431 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
@@ -134,12 +134,12 @@ public class HepPlanner extends AbstractRelOptPlanner {
     this.mainProgram = program;
     this.onCopyHook =
         Util.first(onCopyHook, Functions.<RelNode, RelNode, Void>ignore2());
-    mapDigestToVertex = new HashMap<String, HepRelVertex>();
+    mapDigestToVertex = new HashMap<>();
     graph = DefaultDirectedGraph.create();
 
     // NOTE jvs 24-Apr-2006:  We use LinkedHashSet here and below
     // in order to provide deterministic behavior.
-    allRules = new LinkedHashSet<RelOptRule>();
+    allRules = new LinkedHashSet<>();
     this.noDAG = noDAG;
   }
 
@@ -267,7 +267,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
       LOGGER.finest("Applying rule class " + instruction.ruleClass);
     }
     if (instruction.ruleSet == null) {
-      instruction.ruleSet = new LinkedHashSet<RelOptRule>();
+      instruction.ruleSet = new LinkedHashSet<>();
       for (RelOptRule rule : allRules) {
         if (instruction.ruleClass.isInstance(rule)) {
           instruction.ruleSet.add(rule);
@@ -299,7 +299,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
       HepInstruction.ConverterRules instruction) {
     assert currentProgram.group == null;
     if (instruction.ruleSet == null) {
-      instruction.ruleSet = new LinkedHashSet<RelOptRule>();
+      instruction.ruleSet = new LinkedHashSet<>();
       for (RelOptRule rule : allRules) {
         if (!(rule instanceof ConverterRule)) {
           continue;
@@ -323,7 +323,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
   void executeInstruction(HepInstruction.CommonRelSubExprRules instruction) {
     assert currentProgram.group == null;
     if (instruction.ruleSet == null) {
-      instruction.ruleSet = new LinkedHashSet<RelOptRule>();
+      instruction.ruleSet = new LinkedHashSet<>();
       for (RelOptRule rule : allRules) {
         if (!(rule instanceof CommonRelSubExprRule)) {
           continue;
@@ -447,7 +447,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
     // TODO jvs 4-Apr-2006:  enhance TopologicalOrderIterator
     // to support reverse walk.
     assert currentProgram.matchOrder == HepMatchOrder.BOTTOM_UP;
-    final List<HepRelVertex> list = new ArrayList<HepRelVertex>();
+    final List<HepRelVertex> list = new ArrayList<>();
     for (HepRelVertex vertex : iter) {
       list.add(vertex);
     }
@@ -479,15 +479,14 @@ public class HepPlanner extends AbstractRelOptPlanner {
       if (parentVertices.size() < 2) {
         return null;
       }
-      parents = new ArrayList<RelNode>();
+      parents = new ArrayList<>();
       for (HepRelVertex pVertex : parentVertices) {
         parents.add(pVertex.getCurrentRel());
       }
     }
 
-    List<RelNode> bindings = new ArrayList<RelNode>();
-    Map<RelNode, List<RelNode>> nodeChildren =
-        new HashMap<RelNode, List<RelNode>>();
+    final List<RelNode> bindings = new ArrayList<>();
+    final Map<RelNode, List<RelNode>> nodeChildren = new HashMap<>();
     boolean match =
         matchOperands(
             rule.getOperand(),
@@ -554,8 +553,8 @@ public class HepPlanner extends AbstractRelOptPlanner {
    * @return the list of parents for the vertex
    */
   private List<HepRelVertex> getVertexParents(HepRelVertex vertex) {
-    List<HepRelVertex> parents = new ArrayList<HepRelVertex>();
-    List<HepRelVertex> parentVertices =
+    final List<HepRelVertex> parents = new ArrayList<>();
+    final List<HepRelVertex> parentVertices =
         Graphs.predecessorListOf(graph, vertex);
 
     for (HepRelVertex pVertex : parentVertices) {
@@ -604,7 +603,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
           return false;
         }
       }
-      List<RelNode> children = new ArrayList<RelNode>(childRels.size());
+      final List<RelNode> children = new ArrayList<>(childRels.size());
       for (HepRelVertex childRel : childRels) {
         children.add(childRel.getCurrentRel());
       }
@@ -649,13 +648,13 @@ public class HepPlanner extends AbstractRelOptPlanner {
       bestRel = call.getResults().get(0);
     } else {
       RelOptCost bestCost = null;
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
       for (RelNode rel : call.getResults()) {
-        RelOptCost thisCost = getCost(rel);
+        RelOptCost thisCost = getCost(rel, mq);
         if (LOGGER.isLoggable(Level.FINER)) {
-          LOGGER.finer(
-              "considering " + rel + " with cumulative cost="
-              + thisCost + " and rowcount="
-              + RelMetadataQuery.getRowCount(rel));
+          LOGGER.finer("considering " + rel
+              + " with cumulative cost=" + thisCost
+              + " and rowcount=" + mq.getRowCount(rel));
         }
         if ((bestRel == null) || thisCost.isLt(bestCost)) {
           bestRel = rel;
@@ -675,8 +674,9 @@ public class HepPlanner extends AbstractRelOptPlanner {
     // we only update the existing parents, not the new parents
     // (otherwise loops can result).  Also take care of filtering
     // out parents by traits in case we're dealing with a converter rule.
-    List<HepRelVertex> allParents = Graphs.predecessorListOf(graph, vertex);
-    List<HepRelVertex> parents = new ArrayList<HepRelVertex>();
+    final List<HepRelVertex> allParents =
+        Graphs.predecessorListOf(graph, vertex);
+    final List<HepRelVertex> parents = new ArrayList<>();
     for (HepRelVertex parent : allParents) {
       if (parentTrait != null) {
         RelNode parentRel = parent.getCurrentRel();
@@ -758,8 +758,8 @@ public class HepPlanner extends AbstractRelOptPlanner {
 
     // Recursively add children, replacing this rel's inputs
     // with corresponding child vertices.
-    List<RelNode> inputs = rel.getInputs();
-    List<RelNode> newInputs = new ArrayList<RelNode>();
+    final List<RelNode> inputs = rel.getInputs();
+    final List<RelNode> newInputs = new ArrayList<>();
     for (RelNode input1 : inputs) {
       HepRelVertex childVertex = addRelToGraph(input1);
       newInputs.add(childVertex);
@@ -895,7 +895,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
     LOGGER.finest("collecting garbage");
 
     // Yer basic mark-and-sweep.
-    Set<HepRelVertex> rootSet = new HashSet<HepRelVertex>();
+    final Set<HepRelVertex> rootSet = new HashSet<>();
     if (graph.vertexSet().contains(root)) {
       BreadthFirstIterator.reachable(rootSet, graph, root);
     }
@@ -904,7 +904,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
       // Everything is reachable:  no garbage to collect.
       return;
     }
-    Set<HepRelVertex> sweepSet = new HashSet<HepRelVertex>();
+    final Set<HepRelVertex> sweepSet = new HashSet<>();
     for (HepRelVertex vertex : graph.vertexSet()) {
       if (!rootSet.contains(vertex)) {
         sweepSet.add(vertex);
@@ -929,8 +929,8 @@ public class HepPlanner extends AbstractRelOptPlanner {
 
   private void assertNoCycles() {
     // Verify that the graph is acyclic.
-    CycleDetector<HepRelVertex, DefaultEdge> cycleDetector =
-        new CycleDetector<HepRelVertex, DefaultEdge>(graph);
+    final CycleDetector<HepRelVertex, DefaultEdge> cycleDetector =
+        new CycleDetector<>(graph);
     Set<HepRelVertex> cyclicVertices = cycleDetector.findCycles();
     if (cyclicVertices.isEmpty()) {
       return;
@@ -948,6 +948,7 @@ public class HepPlanner extends AbstractRelOptPlanner {
 
     assertNoCycles();
 
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final StringBuilder sb = new StringBuilder();
     sb.append("\nBreadth-first from root:  {\n");
     for (HepRelVertex vertex : BreadthFirstIterator.of(graph, root)) {
@@ -957,9 +958,9 @@ public class HepPlanner extends AbstractRelOptPlanner {
       RelNode rel = vertex.getCurrentRel();
       sb.append(rel)
           .append(", rowcount=")
-          .append(RelMetadataQuery.getRowCount(rel))
+          .append(mq.getRowCount(rel))
           .append(", cumulative cost=")
-          .append(getCost(rel))
+          .append(getCost(rel, mq))
           .append('\n');
     }
     sb.append("}");

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java
index e944dbe..3f1170a 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepRelMetadataProvider.java
@@ -19,8 +19,8 @@ package org.apache.calcite.plan.hep;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.Metadata;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
-
-import com.google.common.base.Function;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.metadata.UnboundMetadata;
 
 /**
  * HepRelMetadataProvider implements the {@link RelMetadataProvider} interface
@@ -29,20 +29,20 @@ import com.google.common.base.Function;
 class HepRelMetadataProvider implements RelMetadataProvider {
   //~ Methods ----------------------------------------------------------------
 
-  public Function<RelNode, Metadata> apply(Class<? extends RelNode> relClass,
-      final Class<? extends Metadata> metadataClass) {
-    return new Function<RelNode, Metadata>() {
-      public Metadata apply(RelNode rel) {
+  public <M extends Metadata> UnboundMetadata<M>
+  apply(Class<? extends RelNode> relClass,
+      final Class<? extends M> metadataClass) {
+    return new UnboundMetadata<M>() {
+      public M bind(RelNode rel, RelMetadataQuery mq) {
         if (!(rel instanceof HepRelVertex)) {
           return null;
         }
-
         HepRelVertex vertex = (HepRelVertex) rel;
         final RelNode rel2 = vertex.getCurrentRel();
-        Function<RelNode, Metadata> function =
-            rel.getCluster().getMetadataProvider().apply(
-                rel2.getClass(), metadataClass);
-        return function.apply(rel2);
+        UnboundMetadata<M> function =
+            rel.getCluster().getMetadataProvider().apply(rel2.getClass(),
+                metadataClass);
+        return function.bind(rel2, mq);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java b/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
index 15e8894..249d37d 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
@@ -60,14 +60,15 @@ public class HepRelVertex extends AbstractRelNode {
     return this;
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // HepRelMetadataProvider is supposed to intercept this
     // and redirect to the real rels. But sometimes it doesn't.
     return planner.getCostFactory().makeTinyCost();
   }
 
-  @Override public double getRows() {
-    return RelMetadataQuery.getRowCount(currentRel);
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
+    return mq.getRowCount(currentRel);
   }
 
   @Override protected RelDataType deriveRowType() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java b/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
index 96b7c19..42604c9 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
@@ -27,6 +27,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 import java.util.List;
 
@@ -66,7 +67,7 @@ public class AbstractConverter extends ConverterImpl {
         traitSet);
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
     return planner.getCostFactory().makeInfiniteCost();
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
index cf78dff..35cf026 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
@@ -23,6 +23,7 @@ import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
@@ -275,12 +276,12 @@ class RelSet {
     }
 
     // Make sure the cost changes as a result of merging are propagated.
-    Set<RelSubset> activeSet = new HashSet<>();
+    final Set<RelSubset> activeSet = new HashSet<>();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     for (RelNode parentRel : getParentRels()) {
       final RelSubset parentSubset = planner.getSubset(parentRel);
       parentSubset.propagateCostImprovements(
-          planner,
-          parentRel,
+          planner, mq, parentRel,
           activeSet);
     }
     assert activeSet.isEmpty();

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
index 3fede02..b71390b 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
@@ -128,8 +128,9 @@ public class RelSubset extends AbstractRelNode {
    */
   private void computeBestCost(RelOptPlanner planner) {
     bestCost = planner.getCostFactory().makeInfiniteCost();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     for (RelNode rel : getRels()) {
-      final RelOptCost cost = planner.getCost(rel);
+      final RelOptCost cost = planner.getCost(rel, mq);
       if (cost.isLt(bestCost)) {
         bestCost = cost;
         best = rel;
@@ -141,19 +142,23 @@ public class RelSubset extends AbstractRelNode {
     return best;
   }
 
+  public RelNode getOriginal() {
+    return set.rel;
+  }
+
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     throw new UnsupportedOperationException();
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
     return planner.getCostFactory().makeZeroCost();
   }
 
-  public double getRows() {
+  public double estimateRowCount(RelMetadataQuery mq) {
     if (best != null) {
-      return RelMetadataQuery.getRowCount(best);
+      return mq.getRowCount(best);
     } else {
-      return RelMetadataQuery.getRowCount(set.rel);
+      return mq.getRowCount(set.rel);
     }
   }
 
@@ -189,7 +194,7 @@ public class RelSubset extends AbstractRelNode {
    * subset.
    */
   Set<RelNode> getParents() {
-    final Set<RelNode> list = new LinkedHashSet<RelNode>();
+    final Set<RelNode> list = new LinkedHashSet<>();
     for (RelNode parent : set.getParentRels()) {
       for (RelSubset rel : inputSubsets(parent)) {
         if (rel.set == set && traitSet.satisfies(rel.getTraitSet())) {
@@ -205,7 +210,7 @@ public class RelSubset extends AbstractRelNode {
    * of whose inputs is in this subset.
    */
   Set<RelSubset> getParentSubsets(VolcanoPlanner planner) {
-    final Set<RelSubset> list = new LinkedHashSet<RelSubset>();
+    final Set<RelSubset> list = new LinkedHashSet<>();
     for (RelNode parent : set.getParentRels()) {
       for (RelSubset rel : inputSubsets(parent)) {
         if (rel.set == set && rel.getTraitSet().equals(traitSet)) {
@@ -226,7 +231,7 @@ public class RelSubset extends AbstractRelNode {
    * subset. The elements of the list are distinct.
    */
   public Collection<RelNode> getParentRels() {
-    final Set<RelNode> list = new LinkedHashSet<RelNode>();
+    final Set<RelNode> list = new LinkedHashSet<>();
   parentLoop:
     for (RelNode parent : set.getParentRels()) {
       for (RelSubset rel : inputSubsets(parent)) {
@@ -303,24 +308,21 @@ public class RelSubset extends AbstractRelNode {
    * recursively checks whether that subset's parents have gotten cheaper.
    *
    * @param planner   Planner
+   * @param mq        Metadata query
    * @param rel       Relational expression whose cost has improved
    * @param activeSet Set of active subsets, for cycle detection
    */
-  void propagateCostImprovements(
-      VolcanoPlanner planner,
-      RelNode rel,
-      Set<RelSubset> activeSet) {
+  void propagateCostImprovements(VolcanoPlanner planner, RelMetadataQuery mq,
+      RelNode rel, Set<RelSubset> activeSet) {
     for (RelSubset subset : set.subsets) {
       if (rel.getTraitSet().satisfies(subset.traitSet)) {
-        subset.propagateCostImprovements0(planner, rel, activeSet);
+        subset.propagateCostImprovements0(planner, mq, rel, activeSet);
       }
     }
   }
 
-  void propagateCostImprovements0(
-      VolcanoPlanner planner,
-      RelNode rel,
-      Set<RelSubset> activeSet) {
+  void propagateCostImprovements0(VolcanoPlanner planner, RelMetadataQuery mq,
+      RelNode rel, Set<RelSubset> activeSet) {
     ++timestamp;
 
     if (!activeSet.add(this)) {
@@ -331,7 +333,7 @@ public class RelSubset extends AbstractRelNode {
       return;
     }
     try {
-      final RelOptCost cost = planner.getCost(rel);
+      final RelOptCost cost = planner.getCost(rel, mq);
       if (cost.isLt(bestCost)) {
         if (LOGGER.isLoggable(Level.FINER)) {
           LOGGER.finer("Subset cost improved: subset [" + this
@@ -346,8 +348,8 @@ public class RelSubset extends AbstractRelNode {
         planner.ruleQueue.recompute(this);
         for (RelNode parent : getParents()) {
           final RelSubset parentSubset = planner.getSubset(parent);
-          parentSubset.propagateCostImprovements(
-              planner, parent, activeSet);
+          parentSubset.propagateCostImprovements(planner, mq, parent,
+              activeSet);
         }
         planner.checkForSatisfiedConverters(set, rel);
       }
@@ -401,7 +403,7 @@ public class RelSubset extends AbstractRelNode {
    * As {@link #getRels()} but returns a list.
    */
   public List<RelNode> getRelList() {
-    final List<RelNode> list = new ArrayList<RelNode>();
+    final List<RelNode> list = new ArrayList<>();
     for (RelNode rel : set.rels) {
       if (rel.getTraitSet().satisfies(traitSet)) {
         list.add(rel);
@@ -460,7 +462,7 @@ public class RelSubset extends AbstractRelNode {
       }
 
       List<RelNode> oldInputs = p.getInputs();
-      List<RelNode> inputs = new ArrayList<RelNode>();
+      List<RelNode> inputs = new ArrayList<>();
       for (int i = 0; i < oldInputs.size(); i++) {
         RelNode oldInput = oldInputs.get(i);
         RelNode input = visit(oldInput, i, p);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java b/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java
index bce60ab..f14a681 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RuleQueue.java
@@ -20,6 +20,7 @@ import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelNodes;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.ChunkList;
 import org.apache.calcite.util.Stacks;
 import org.apache.calcite.util.Util;
@@ -392,12 +393,14 @@ class RuleQueue {
       // The root always has importance = 1
       importance = 1.0;
     } else {
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
+
       // The importance of a subset is the max of its importance to its
       // parents
       importance = 0.0;
       for (RelSubset parent : subset.getParentSubsets(planner)) {
         final double childImportance =
-            computeImportanceOfChild(subset, parent);
+            computeImportanceOfChild(mq, subset, parent);
         importance = Math.max(importance, childImportance);
       }
     }
@@ -582,12 +585,11 @@ class RuleQueue {
    * with cost 50 will have importance 0.4, and a child with cost 25 will have
    * importance 0.2.
    */
-  private double computeImportanceOfChild(
-      RelSubset child,
+  private double computeImportanceOfChild(RelMetadataQuery mq, RelSubset child,
       RelSubset parent) {
     final double parentImportance = getImportance(parent);
-    final double childCost = toDouble(planner.getCost(child));
-    final double parentCost = toDouble(planner.getCost(parent));
+    final double childCost = toDouble(planner.getCost(child, mq));
+    final double parentCost = toDouble(planner.getCost(parent, mq));
     double alpha = childCost / parentCost;
     if (alpha >= 1.0) {
       // child is always less important than parent
@@ -688,7 +690,7 @@ class RuleQueue {
      * <p>Use a hunkList because {@link java.util.ArrayList} does not implement
      * remove(0) efficiently.</p>
      */
-    final List<VolcanoRuleMatch> list = new ChunkList<VolcanoRuleMatch>();
+    final List<VolcanoRuleMatch> list = new ChunkList<>();
 
     /**
      * A set of rule-match names contained in {@link #list}. Allows fast

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
index 425ebec..530148c 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
@@ -950,8 +950,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     RelVisitor visitor =
         new RelVisitor() {
           int depth = 0;
-
-          final HashSet<RelSubset> visitedSubsets = new HashSet<>();
+          final Set<RelSubset> visitedSubsets = new HashSet<>();
 
           public void visit(
               RelNode p,
@@ -991,7 +990,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * {@link Convention#NONE} and boosts their importance by 25%.
    */
   private void injectImportanceBoost() {
-    final HashSet<RelSubset> requireBoost = new HashSet<>();
+    final Set<RelSubset> requireBoost = new HashSet<>();
 
   SUBSET_LOOP:
     for (RelSubset subset : ruleQueue.subsetImportances.keySet()) {
@@ -1060,6 +1059,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * Checks internal consistency.
    */
   protected void validate() {
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     for (RelSet set : allSets) {
       if (set.equivalentSet != null) {
         throw new AssertionError(
@@ -1073,7 +1073,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
               + "] is in wrong set [" + set + "]");
         }
         for (RelNode rel : subset.getRels()) {
-          RelOptCost relCost = getCost(rel);
+          RelOptCost relCost = getCost(rel, mq);
           if (relCost.isLt(subset.bestCost)) {
             throw new AssertionError(
                 "rel [" + rel.getDescription()
@@ -1118,7 +1118,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     }
   }
 
-  public RelOptCost getCost(RelNode rel) {
+  public RelOptCost getCost(RelNode rel, RelMetadataQuery mq) {
     assert rel != null : "pre-condition: rel != null";
     if (rel instanceof RelSubset) {
       return ((RelSubset) rel).bestCost;
@@ -1127,13 +1127,13 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
         == Convention.NONE) {
       return costFactory.makeInfiniteCost();
     }
-    RelOptCost cost = RelMetadataQuery.getNonCumulativeCost(rel);
+    RelOptCost cost = mq.getNonCumulativeCost(rel);
     if (!zeroCost.isLt(cost)) {
       // cost must be positive, so nudge it
       cost = costFactory.makeTinyCost();
     }
     for (RelNode input : rel.getInputs()) {
-      cost = cost.plus(getCost(input));
+      cost = cost.plus(getCost(input, mq));
     }
     return cost;
   }
@@ -1341,6 +1341,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * @see #normalizePlan(String)
    */
   public void dump(PrintWriter pw) {
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     pw.println("Root: " + root.getDescription());
     pw.println("Original rel:");
     pw.println(originalRootString);
@@ -1395,8 +1396,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
           if (importance != null) {
             pw.print(", importance=" + importance);
           }
-          pw.print(", rowcount=" + RelMetadataQuery.getRowCount(rel));
-          pw.println(", cumulative cost=" + getCost(rel));
+          pw.print(", rowcount=" + mq.getRowCount(rel));
+          pw.println(", cumulative cost=" + getCost(rel, mq));
         }
       }
     }
@@ -1657,6 +1658,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     // implements the interface required by its calling convention.
     final RelTraitSet traits = rel.getTraitSet();
     final Convention convention = traits.getTrait(ConventionTraitDef.INSTANCE);
+    assert convention != null;
     if (!convention.getInterface().isInstance(rel)
         && !(rel instanceof Converter)) {
       throw Util.newInternal(
@@ -1842,7 +1844,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     // 100. We think this happens because the back-links to parents are
     // not established. So, give the subset another change to figure out
     // its cost.
-    subset.propagateCostImprovements(this, rel, new HashSet<RelSubset>());
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    subset.propagateCostImprovements(this, mq, rel, new HashSet<RelSubset>());
 
     return subset;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java
index e898a61..201670f 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRelMetadataProvider.java
@@ -19,8 +19,8 @@ package org.apache.calcite.plan.volcano;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.Metadata;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
-
-import com.google.common.base.Function;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.metadata.UnboundMetadata;
 
 /**
  * VolcanoRelMetadataProvider implements the {@link RelMetadataProvider}
@@ -29,16 +29,19 @@ import com.google.common.base.Function;
 public class VolcanoRelMetadataProvider implements RelMetadataProvider {
   //~ Methods ----------------------------------------------------------------
 
-  public Function<RelNode, Metadata> apply(Class<? extends RelNode> relClass,
-      final Class<? extends Metadata> metadataClass) {
+  public <M extends Metadata> UnboundMetadata<M>
+  apply(Class<? extends RelNode> relClass,
+      final Class<? extends M> metadataClass) {
     if (relClass != RelSubset.class) {
       // let someone else further down the chain sort it out
       return null;
     }
 
-    return new Function<RelNode, Metadata>() {
-      public Metadata apply(RelNode rel) {
-        RelSubset subset = (RelSubset) rel;
+    return new UnboundMetadata<M>() {
+      public M bind(RelNode rel, RelMetadataQuery mq) {
+        final RelSubset subset = (RelSubset) rel;
+        final RelMetadataProvider provider =
+            rel.getCluster().getMetadataProvider();
 
         // REVIEW jvs 29-Mar-2006: I'm not sure what the correct precedence
         // should be here.  Letting the current best plan take the first shot is
@@ -49,11 +52,10 @@ public class VolcanoRelMetadataProvider implements RelMetadataProvider {
         // First, try current best implementation.  If it knows how to answer
         // this query, treat it as the most reliable.
         if (subset.best != null) {
-          final Function<RelNode, Metadata> function =
-              rel.getCluster().getMetadataProvider().apply(
-                  subset.best.getClass(), metadataClass);
+          final UnboundMetadata<M> function =
+              provider.apply(subset.best.getClass(), metadataClass);
           if (function != null) {
-            Metadata metadata = function.apply(subset.best);
+            final M metadata = function.bind(subset.best, mq);
             if (metadata != null) {
               return metadata;
             }
@@ -79,11 +81,10 @@ public class VolcanoRelMetadataProvider implements RelMetadataProvider {
         subset.set.inMetadataQuery = true;
         try {
           for (RelNode relCandidate : subset.set.rels) {
-            final Function<RelNode, Metadata> function =
-                rel.getCluster().getMetadataProvider().apply(
-                    relCandidate.getClass(), metadataClass);
+            final UnboundMetadata<M> function =
+                provider.apply(relCandidate.getClass(), metadataClass);
             if (function != null) {
-              final Metadata result = function.apply(relCandidate);
+              final M result = function.bind(relCandidate, mq);
               if (result != null) {
                 return result;
               }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index d58cfd8..d75d9c9 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -26,6 +26,7 @@ import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.schema.SchemaPlus;
@@ -269,6 +270,10 @@ public class PlannerImpl implements Planner {
   public RelNode transform(int ruleSetIndex, RelTraitSet requiredOutputTraits,
       RelNode rel) throws RelConversionException {
     ensure(State.STATE_5_CONVERTED);
+    rel.getCluster().setMetadataProvider(
+        new CachingRelMetadataProvider(
+            rel.getCluster().getMetadataProvider(),
+            rel.getCluster().getPlanner()));
     Program program = programs.get(ruleSetIndex);
     return program.run(planner, rel, requiredOutputTraits);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
index 75a546c..164d34e 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -29,6 +29,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.externalize.RelWriterImpl;
 import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rel.metadata.MetadataFactory;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
@@ -162,11 +163,13 @@ public abstract class AbstractRelNode implements RelNode {
   }
 
   public boolean isDistinct() {
-    return Boolean.TRUE.equals(RelMetadataQuery.areRowsUnique(this));
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return Boolean.TRUE.equals(mq.areRowsUnique(this));
   }
 
   public boolean isKey(ImmutableBitSet columns) {
-    return Boolean.TRUE.equals(RelMetadataQuery.areColumnsUnique(this, columns));
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return Boolean.TRUE.equals(mq.areColumnsUnique(this, columns));
   }
 
   public int getId() {
@@ -235,7 +238,11 @@ public abstract class AbstractRelNode implements RelNode {
     return Collections.emptyList();
   }
 
-  public double getRows() {
+  public final double getRows() {
+    return estimateRowCount(RelMetadataQuery.instance());
+  }
+
+  public double estimateRowCount(RelMetadataQuery mq) {
     return 1.0;
   }
 
@@ -271,15 +278,22 @@ public abstract class AbstractRelNode implements RelNode {
     return this;
   }
 
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  public final RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return computeSelfCost(planner, RelMetadataQuery.instance());
+  }
+
+  public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // by default, assume cost is proportional to number of rows
-    double rowCount = RelMetadataQuery.getRowCount(this);
+    double rowCount = mq.getRowCount(this);
     double bytesPerRow = 1;
     return planner.getCostFactory().makeCost(rowCount, rowCount, 0);
   }
 
-  public final <M extends Metadata> M metadata(Class<M> metadataClass) {
-    final M metadata = cluster.getMetadataFactory().query(this, metadataClass);
+  public final <M extends Metadata> M metadata(Class<M> metadataClass,
+      RelMetadataQuery mq) {
+    final MetadataFactory factory = cluster.getMetadataFactory();
+    final M metadata = factory.query(this, mq, metadataClass);
     assert metadata != null
         : "no provider found (rel=" + this + ", m=" + metadataClass
         + "); a backstop provider is recommended";

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/RelNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelNode.java b/core/src/main/java/org/apache/calcite/rel/RelNode.java
index 973bc98..0108e88 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelNode.java
@@ -170,9 +170,18 @@ public interface RelNode extends RelOptNode, Cloneable {
    * {@link RelMetadataQuery#getRowCount}, which gives plugins a chance to
    * override the rel's default ideas about row count.
    *
+   * @param mq Metadata query
    * @return Estimate of the number of rows this relational expression will
    *   return
    */
+  double estimateRowCount(RelMetadataQuery mq);
+
+  /**
+   * @deprecated Call {@link RelMetadataQuery#getRowCount(RelNode)};
+   * if you wish to override the default row count formula, override the
+   * {@link #estimateRowCount(RelMetadataQuery)} method.
+   */
+  @Deprecated // to be removed before 2.0
   double getRows();
 
   /**
@@ -246,20 +255,31 @@ public interface RelNode extends RelOptNode, Cloneable {
    * chance to override the rel's default ideas about cost.
    *
    * @param planner Planner for cost calculation
+   * @param mq Metadata query
    * @return Cost of this plan (not including children)
    */
+  RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq);
+
+  /**
+   * @deprecated Call {@link RelMetadataQuery#getNonCumulativeCost(RelNode)};
+   * if you wish to override the default cost formula, override the
+   * {@link #computeSelfCost(RelOptPlanner, RelMetadataQuery)} method.
+   */
+  @Deprecated // to be removed before 2.0
   RelOptCost computeSelfCost(RelOptPlanner planner);
 
   /**
    * Returns a metadata interface.
    *
-   * @param metadataClass Metadata interface
    * @param <M> Type of metadata being requested
+   * @param metadataClass Metadata interface
+   * @param mq Metadata query
+   *
    * @return Metadata object that supplies the desired metadata (never null,
    *     although if the information is not present the metadata object may
    *     return null from all methods)
    */
-  <M extends Metadata> M metadata(Class<M> metadataClass);
+  <M extends Metadata> M metadata(Class<M> metadataClass, RelMetadataQuery mq);
 
   /**
    * Describes the inputs and attributes of this relational expression.

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/SingleRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/SingleRel.java b/core/src/main/java/org/apache/calcite/rel/SingleRel.java
index 5fbb2c6..186db36 100644
--- a/core/src/main/java/org/apache/calcite/rel/SingleRel.java
+++ b/core/src/main/java/org/apache/calcite/rel/SingleRel.java
@@ -63,9 +63,9 @@ public abstract class SingleRel extends AbstractRelNode {
     return ImmutableList.of(input);
   }
 
-  @Override public double getRows() {
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
     // Not necessarily correct, but a better default than AbstractRelNode's 1.0
-    return RelMetadataQuery.getRowCount(input);
+    return mq.getRowCount(input);
   }
 
   @Override public void childrenAccept(RelVisitor visitor) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java b/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java
index 4bb84be..b0cb40b 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java
@@ -58,9 +58,9 @@ public abstract class ConverterImpl extends SingleRel
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelNode
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    double dRows = RelMetadataQuery.getRowCount(getInput());
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    double dRows = mq.getRowCount(getInput());
     double dCpu = dRows;
     double dIo = 0;
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 1a07bed..7b46940 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
@@ -276,7 +276,7 @@ public abstract class Aggregate extends SingleRel {
     return pw;
   }
 
-  @Override public double getRows() {
+  @Override public double estimateRowCount(RelMetadataQuery mq) {
     // Assume that each sort column has 50% of the value count.
     // Therefore one sort column has .5 * rowCount,
     // 2 sort columns give .75 * rowCount.
@@ -285,16 +285,17 @@ public abstract class Aggregate extends SingleRel {
     if (groupCount == 0) {
       return 1;
     } else {
-      double rowCount = super.getRows();
+      double rowCount = super.estimateRowCount(mq);
       rowCount *= 1.0 - Math.pow(.5, groupCount);
       return rowCount;
     }
   }
 
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
     // REVIEW jvs 24-Aug-2008:  This is bogus, but no more bogus
     // than what's currently in Join.
-    double rowCount = RelMetadataQuery.getRowCount(this);
+    double rowCount = mq.getRowCount(this);
     // Aggregates with more aggregate functions cost a bit more
     float multiplier = 1f + (float) aggCalls.size() * 0.125f;
     for (AggregateCall aggCall : aggCalls) {


[15/50] [abbrv] calcite git commit: [CALCITE-955] Litmus (continuation-passing style for methods that check invariants)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
index ced121c..a4150c4 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -478,10 +479,11 @@ public class SqlWindow extends SqlCall {
    * definition.
    *
    * @param node The SqlWindow to compare to "this" window
-   * @param fail Whether to throw if not equal
+   * @param litmus What to do if an error is detected (nodes are not equal)
+   *
    * @return boolean true if all nodes in the subtree are equal
    */
-  @Override public boolean equalsDeep(SqlNode node, boolean fail) {
+  @Override public boolean equalsDeep(SqlNode node, Litmus litmus) {
     // This is the difference over super.equalsDeep.  It skips
     // operands[0] the declared name fo this window.  We only want
     // to check the window components.
@@ -489,8 +491,7 @@ public class SqlWindow extends SqlCall {
         || node instanceof SqlWindow
         && SqlNode.equalDeep(
             Util.skip(getOperandList()),
-            Util.skip(((SqlWindow) node).getOperandList()),
-          fail);
+            Util.skip(((SqlWindow) node).getOperandList()), litmus);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
index 547e0e7..9e8105a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
@@ -34,6 +34,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 
 import com.google.common.collect.ImmutableList;
 
@@ -84,6 +85,13 @@ public class SqlInOperator extends SqlBinaryOperator {
     return isNotIn;
   }
 
+  @Override public boolean validRexOperands(int count, Litmus litmus) {
+    if (count == 0) {
+      return litmus.fail("wrong operand count " + count + " for " + this);
+    }
+    return litmus.succeed();
+  }
+
   public RelDataType deriveType(
       SqlValidator validator,
       SqlValidatorScope scope,
@@ -100,7 +108,7 @@ public class SqlInOperator extends SqlBinaryOperator {
     // Derive type for RHS.
     if (right instanceof SqlNodeList) {
       // Handle the 'IN (expr, ...)' form.
-      List<RelDataType> rightTypeList = new ArrayList<RelDataType>();
+      List<RelDataType> rightTypeList = new ArrayList<>();
       SqlNodeList nodeList = (SqlNodeList) right;
       for (int i = 0; i < nodeList.size(); i++) {
         SqlNode node = nodeList.get(i);

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/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 3311f50..ec26b7c 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
@@ -49,6 +49,7 @@ import org.apache.calcite.sql.type.SqlOperandCountRanges;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlModality;
+import org.apache.calcite.util.Litmus;
 
 /**
  * Implementation of {@link org.apache.calcite.sql.SqlOperatorTable} containing
@@ -613,6 +614,13 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
         public boolean argumentMustBeScalar(int ordinal) {
           return false;
         }
+
+        @Override public boolean validRexOperands(int count, Litmus litmus) {
+          if (count != 0) {
+            return litmus.fail("wrong operand count " + count + " for " + this);
+          }
+          return litmus.succeed();
+        }
       };
 
   public static final SqlPrefixOperator NOT =

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/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 3b402af..e4e5edf 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql2rel;
 
+import org.apache.calcite.avatica.util.Spaces;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
@@ -146,6 +147,7 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.NumberUtil;
 import org.apache.calcite.util.Pair;
@@ -453,8 +455,8 @@ public class SqlToRelConverter {
         validator.getTypeFactory().createStructType(convertedFields);
 
     if (!RelOptUtil.equal("validated row type", validatedRowType,
-        "converted row type", convertedRowType, false)) {
-      throw Util.newInternal("Conversion to relational algebra failed to "
+        "converted row type", convertedRowType, Litmus.IGNORE)) {
+      throw new AssertionError("Conversion to relational algebra failed to "
           + "preserve datatypes:\n"
           + "validated type:\n"
           + validatedRowType.getFullTypeString()
@@ -1643,7 +1645,7 @@ public class SqlToRelConverter {
       NlsString unpadded = (NlsString) value;
       return rexBuilder.makeCharLiteral(
           new NlsString(
-              Util.rpad(unpadded.getValue(), type.getPrecision()),
+              Spaces.padRight(unpadded.getValue(), type.getPrecision()),
               unpadded.getCharsetName(),
               unpadded.getCollation()));
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/util/Litmus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Litmus.java b/core/src/main/java/org/apache/calcite/util/Litmus.java
index e7e2bbd..1a74e6c 100644
--- a/core/src/main/java/org/apache/calcite/util/Litmus.java
+++ b/core/src/main/java/org/apache/calcite/util/Litmus.java
@@ -23,13 +23,37 @@ public interface Litmus {
   /** Implementation of {@link org.apache.calcite.util.Litmus} that throws
    * an {@link java.lang.AssertionError} on failure. */
   Litmus THROW = new Litmus() {
-    @Override public boolean fail(String message) {
+    public boolean fail(String message) {
       throw new AssertionError(message);
     }
 
-    @Override public boolean succeed() {
+    public boolean succeed() {
       return true;
     }
+
+    public boolean check(boolean condition, Object info) {
+      if (condition) {
+        return succeed();
+      } else {
+        return fail(info == null ? null : info.toString());
+      }
+    }
+  };
+
+  /** Implementation of {@link org.apache.calcite.util.Litmus} that returns
+   * a status code but does not throw. */
+  Litmus IGNORE = new Litmus() {
+    public boolean fail(String message) {
+      return false;
+    }
+
+    public boolean succeed() {
+      return true;
+    }
+
+    public boolean check(boolean condition, Object info) {
+      return condition;
+    }
   };
 
   /** Called when test fails. Returns false or throws. */
@@ -37,6 +61,14 @@ public interface Litmus {
 
   /** Called when test succeeds. Returns true. */
   boolean succeed();
+
+  /** Checks a condition.
+   *
+   * <p>If the condition is true, calls {@link #succeed};
+   * if the condition is false, calls {@link #fail},
+   * converting {@code info} into a string message.
+   */
+  boolean check(boolean condition, Object info);
 }
 
 // End Litmus.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 4dc2fdc..ed39af0 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -1183,6 +1183,8 @@ public class Util {
    * @param s   string to be padded
    * @param len desired length
    * @return padded string
+   *
+   * @deprecated Use {@link Spaces#padRight(String, int)}
    */
   @Deprecated // to be removed before 2.0
   public static String rpad(String s, int len) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/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 314446e..1c67bc8 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -21,6 +21,7 @@ import org.apache.calcite.rel.RelVisitor;
 import org.apache.calcite.rel.externalize.RelXmlWriter;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
@@ -1457,13 +1458,13 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   /**
    * Visitor that checks that every {@link RelNode} in a tree is valid.
    *
-   * @see RelNode#isValid(boolean)
+   * @see RelNode#isValid(org.apache.calcite.util.Litmus)
    */
   public static class RelValidityChecker extends RelVisitor {
     int invalidCount;
 
     public void visit(RelNode node, int ordinal, RelNode parent) {
-      if (!node.isValid(true)) {
+      if (!node.isValid(Litmus.THROW)) {
         ++invalidCount;
       }
       super.visit(node, ordinal, parent);


[41/50] [abbrv] calcite git commit: Upgrade toolbox, to fix line length issue on Windows

Posted by jh...@apache.org.
Upgrade toolbox, to fix line length issue on Windows


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

Branch: refs/heads/branch-release
Commit: e15f89e5823104f0890ee92675cc0da45dcec4ca
Parents: b4b04d2
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 11 18:49:50 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 12 13:47:17 2016 -0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/e15f89e5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3253f25..2d83845 100644
--- a/pom.xml
+++ b/pom.xml
@@ -400,7 +400,7 @@ limitations under the License.
           <dependency>
             <groupId>net.hydromatic</groupId>
             <artifactId>toolbox</artifactId>
-            <version>0.2</version>
+            <version>0.3</version>
           </dependency>
         </dependencies>
       </plugin>


[30/50] [abbrv] calcite git commit: [CALCITE-794] Detect cycles when computing statistics

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
index d77ca13..cb1de64 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUniqueKeys.java
@@ -54,20 +54,23 @@ public class RelMdUniqueKeys {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Set<ImmutableBitSet> getUniqueKeys(Filter rel, boolean ignoreNulls) {
-    return RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
+  public Set<ImmutableBitSet> getUniqueKeys(Filter rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+    return mq.getUniqueKeys(rel.getInput(), ignoreNulls);
   }
 
-  public Set<ImmutableBitSet> getUniqueKeys(Sort rel, boolean ignoreNulls) {
-    return RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
+  public Set<ImmutableBitSet> getUniqueKeys(Sort rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+    return mq.getUniqueKeys(rel.getInput(), ignoreNulls);
   }
 
-  public Set<ImmutableBitSet> getUniqueKeys(Correlate rel,
+  public Set<ImmutableBitSet> getUniqueKeys(Correlate rel, RelMetadataQuery mq,
       boolean ignoreNulls) {
-    return RelMetadataQuery.getUniqueKeys(rel.getLeft(), ignoreNulls);
+    return mq.getUniqueKeys(rel.getLeft(), ignoreNulls);
   }
 
-  public Set<ImmutableBitSet> getUniqueKeys(Project rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(Project rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
     // LogicalProject maps a set of rows to a different set;
     // Without knowledge of the mapping function(whether it
     // preserves uniqueness), it is only safe to derive uniqueness
@@ -75,11 +78,9 @@ public class RelMdUniqueKeys {
     //
     // Further more, the unique bitset coming from the child needs
     // to be mapped to match the output of the project.
-    Map<Integer, Integer> mapInToOutPos = new HashMap<>();
-
-    List<RexNode> projExprs = rel.getProjects();
-
-    Set<ImmutableBitSet> projUniqueKeySet = new HashSet<>();
+    final Map<Integer, Integer> mapInToOutPos = new HashMap<>();
+    final List<RexNode> projExprs = rel.getProjects();
+    final Set<ImmutableBitSet> projUniqueKeySet = new HashSet<>();
 
     // Build an input to output position map.
     for (int i = 0; i < projExprs.size(); i++) {
@@ -96,7 +97,7 @@ public class RelMdUniqueKeys {
     }
 
     Set<ImmutableBitSet> childUniqueKeySet =
-        RelMetadataQuery.getUniqueKeys(rel.getInput(), ignoreNulls);
+        mq.getUniqueKeys(rel.getInput(), ignoreNulls);
 
     if (childUniqueKeySet != null) {
       // Now add to the projUniqueKeySet the child keys that are fully
@@ -123,7 +124,8 @@ public class RelMdUniqueKeys {
     return projUniqueKeySet;
   }
 
-  public Set<ImmutableBitSet> getUniqueKeys(Join rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(Join rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
     final RelNode left = rel.getLeft();
     final RelNode right = rel.getRight();
 
@@ -136,13 +138,11 @@ public class RelMdUniqueKeys {
     // that is undesirable, use RelMetadataQuery.areColumnsUnique() as
     // an alternative way of getting unique key information.
 
-    Set<ImmutableBitSet> retSet = new HashSet<>();
-    Set<ImmutableBitSet> leftSet =
-        RelMetadataQuery.getUniqueKeys(left, ignoreNulls);
+    final Set<ImmutableBitSet> retSet = new HashSet<>();
+    final Set<ImmutableBitSet> leftSet = mq.getUniqueKeys(left, ignoreNulls);
     Set<ImmutableBitSet> rightSet = null;
 
-    Set<ImmutableBitSet> tmpRightSet =
-        RelMetadataQuery.getUniqueKeys(right, ignoreNulls);
+    final Set<ImmutableBitSet> tmpRightSet = mq.getUniqueKeys(right, ignoreNulls);
     int nFieldsOnLeft = left.getRowType().getFieldCount();
 
     if (tmpRightSet != null) {
@@ -169,12 +169,10 @@ public class RelMdUniqueKeys {
 
     // determine if either or both the LHS and RHS are unique on the
     // equijoin columns
-    Boolean leftUnique =
-        RelMetadataQuery.areColumnsUnique(left, joinInfo.leftSet(),
-            ignoreNulls);
-    Boolean rightUnique =
-        RelMetadataQuery.areColumnsUnique(right, joinInfo.rightSet(),
-            ignoreNulls);
+    final Boolean leftUnique =
+        mq.areColumnsUnique(left, joinInfo.leftSet(), ignoreNulls);
+    final Boolean rightUnique =
+        mq.areColumnsUnique(right, joinInfo.rightSet(), ignoreNulls);
 
     // if the right hand side is unique on its equijoin columns, then we can
     // add the unique keys from left if the left hand side is not null
@@ -197,19 +195,20 @@ public class RelMdUniqueKeys {
     return retSet;
   }
 
-  public Set<ImmutableBitSet> getUniqueKeys(SemiJoin rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(SemiJoin rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
     // only return the unique keys from the LHS since a semijoin only
     // returns the LHS
-    return RelMetadataQuery.getUniqueKeys(rel.getLeft(), ignoreNulls);
+    return mq.getUniqueKeys(rel.getLeft(), ignoreNulls);
   }
 
-  public Set<ImmutableBitSet> getUniqueKeys(Aggregate rel,
+  public Set<ImmutableBitSet> getUniqueKeys(Aggregate rel, RelMetadataQuery mq,
       boolean ignoreNulls) {
     // group by keys form a unique key
     return ImmutableSet.of(rel.getGroupSet());
   }
 
-  public Set<ImmutableBitSet> getUniqueKeys(SetOp rel,
+  public Set<ImmutableBitSet> getUniqueKeys(SetOp rel, RelMetadataQuery mq,
       boolean ignoreNulls) {
     if (!rel.all) {
       return ImmutableSet.of(
@@ -219,7 +218,8 @@ public class RelMdUniqueKeys {
   }
 
   // Catch-all rule when none of the others apply.
-  public Set<ImmutableBitSet> getUniqueKeys(RelNode rel, boolean ignoreNulls) {
+  public Set<ImmutableBitSet> getUniqueKeys(RelNode rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
     // no information available
     return null;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
index dc1ea3e..5435979 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
@@ -26,11 +26,14 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Minus;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlFunction;
@@ -77,13 +80,11 @@ public class RelMdUtil {
    * @param rel the semijoin of interest
    * @return constructed rexnode
    */
-  public static RexNode makeSemiJoinSelectivityRexNode(SemiJoin rel) {
+  public static RexNode makeSemiJoinSelectivityRexNode(RelMetadataQuery mq,
+      SemiJoin rel) {
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
     double selectivity =
-        computeSemiJoinSelectivity(
-            rel.getLeft(),
-            rel.getRight(),
-            rel);
+        computeSemiJoinSelectivity(mq, rel.getLeft(), rel.getRight(), rel);
     RexNode selec =
         rexBuilder.makeApproxLiteral(new BigDecimal(selectivity));
     return rexBuilder.makeCall(ARTIFICIAL_SELECTIVITY_FUNC, selec);
@@ -113,12 +114,10 @@ public class RelMdUtil {
    * @param rel semijoin rel
    * @return calculated selectivity
    */
-  public static double computeSemiJoinSelectivity(SemiJoin rel) {
-    return computeSemiJoinSelectivity(
-        rel.getLeft(),
-        rel.getRight(),
-        rel.getLeftKeys(),
-        rel.getRightKeys());
+  public static double computeSemiJoinSelectivity(RelMetadataQuery mq,
+      SemiJoin rel) {
+    return computeSemiJoinSelectivity(mq, rel.getLeft(), rel.getRight(),
+        rel.getLeftKeys(), rel.getRightKeys());
   }
 
   /**
@@ -132,14 +131,9 @@ public class RelMdUtil {
    * @param rel     semijoin rel
    * @return calculated selectivity
    */
-  public static double computeSemiJoinSelectivity(
-      RelNode factRel,
-      RelNode dimRel,
-      SemiJoin rel) {
-    return computeSemiJoinSelectivity(
-        factRel,
-        dimRel,
-        rel.getLeftKeys(),
+  public static double computeSemiJoinSelectivity(RelMetadataQuery mq,
+      RelNode factRel, RelNode dimRel, SemiJoin rel) {
+    return computeSemiJoinSelectivity(mq, factRel, dimRel, rel.getLeftKeys(),
         rel.getRightKeys());
   }
 
@@ -155,10 +149,8 @@ public class RelMdUtil {
    * @param dimKeyList  RHS keys used in the filter
    * @return calculated selectivity
    */
-  public static double computeSemiJoinSelectivity(
-      RelNode factRel,
-      RelNode dimRel,
-      List<Integer> factKeyList,
+  public static double computeSemiJoinSelectivity(RelMetadataQuery mq,
+      RelNode factRel, RelNode dimRel, List<Integer> factKeyList,
       List<Integer> dimKeyList) {
     ImmutableBitSet.Builder factKeys = ImmutableBitSet.builder();
     for (int factCol : factKeyList) {
@@ -170,20 +162,19 @@ public class RelMdUtil {
     }
     final ImmutableBitSet dimKeys = dimKeyBuilder.build();
 
-    Double factPop =
-        RelMetadataQuery.getPopulationSize(factRel, factKeys.build());
+    Double factPop = mq.getPopulationSize(factRel, factKeys.build());
     if (factPop == null) {
       // use the dimension population if the fact population is
       // unavailable; since we're filtering the fact table, that's
       // the population we ideally want to use
-      factPop = RelMetadataQuery.getPopulationSize(dimRel, dimKeys);
+      factPop = mq.getPopulationSize(dimRel, dimKeys);
     }
 
     // if cardinality and population are available, use them; otherwise
     // use percentage original rows
     Double selectivity;
     Double dimCard =
-        RelMetadataQuery.getDistinctRowCount(
+        mq.getDistinctRowCount(
             dimRel,
             dimKeys,
             null);
@@ -194,7 +185,7 @@ public class RelMdUtil {
       }
       selectivity = dimCard / factPop;
     } else {
-      selectivity = RelMetadataQuery.getPercentageOriginalRows(dimRel);
+      selectivity = mq.getPercentageOriginalRows(dimRel);
     }
 
     if (selectivity == null) {
@@ -220,26 +211,24 @@ public class RelMdUtil {
    * @return true if bit mask represents a unique column set; false if not (or
    * if no metadata is available)
    */
-  public static boolean areColumnsDefinitelyUnique(
-      RelNode rel,
-      ImmutableBitSet colMask) {
-    Boolean b = RelMetadataQuery.areColumnsUnique(rel, colMask, false);
+  public static boolean areColumnsDefinitelyUnique(RelMetadataQuery mq,
+      RelNode rel, ImmutableBitSet colMask) {
+    Boolean b = mq.areColumnsUnique(rel, colMask, false);
     return b != null && b;
   }
 
-  public static Boolean areColumnsUnique(
-      RelNode rel,
+  public static Boolean areColumnsUnique(RelMetadataQuery mq, RelNode rel,
       List<RexInputRef> columnRefs) {
     ImmutableBitSet.Builder colMask = ImmutableBitSet.builder();
     for (RexInputRef columnRef : columnRefs) {
       colMask.set(columnRef.getIndex());
     }
-    return RelMetadataQuery.areColumnsUnique(rel, colMask.build());
+    return mq.areColumnsUnique(rel, colMask.build());
   }
 
-  public static boolean areColumnsDefinitelyUnique(RelNode rel,
-      List<RexInputRef> columnRefs) {
-    Boolean b = areColumnsUnique(rel, columnRefs);
+  public static boolean areColumnsDefinitelyUnique(RelMetadataQuery mq,
+      RelNode rel, List<RexInputRef> columnRefs) {
+    Boolean b = areColumnsUnique(mq, rel, columnRefs);
     return b != null && b;
   }
 
@@ -255,31 +244,29 @@ public class RelMdUtil {
    * @return true if bit mask represents a unique column set; false if not (or
    * if no metadata is available)
    */
-  public static boolean areColumnsDefinitelyUniqueWhenNullsFiltered(RelNode rel,
-      ImmutableBitSet colMask) {
-    Boolean b = RelMetadataQuery.areColumnsUnique(rel, colMask, true);
+  public static boolean areColumnsDefinitelyUniqueWhenNullsFiltered(
+      RelMetadataQuery mq, RelNode rel, ImmutableBitSet colMask) {
+    Boolean b = mq.areColumnsUnique(rel, colMask, true);
     if (b == null) {
       return false;
     }
     return b;
   }
 
-  public static Boolean areColumnsUniqueWhenNullsFiltered(
-      RelNode rel,
-      List<RexInputRef> columnRefs) {
+  public static Boolean areColumnsUniqueWhenNullsFiltered(RelMetadataQuery mq,
+      RelNode rel, List<RexInputRef> columnRefs) {
     ImmutableBitSet.Builder colMask = ImmutableBitSet.builder();
 
     for (RexInputRef columnRef : columnRefs) {
       colMask.set(columnRef.getIndex());
     }
 
-    return RelMetadataQuery.areColumnsUnique(rel, colMask.build(), true);
+    return mq.areColumnsUnique(rel, colMask.build(), true);
   }
 
   public static boolean areColumnsDefinitelyUniqueWhenNullsFiltered(
-      RelNode rel,
-      List<RexInputRef> columnRefs) {
-    Boolean b = areColumnsUniqueWhenNullsFiltered(rel, columnRefs);
+      RelMetadataQuery mq, RelNode rel, List<RexInputRef> columnRefs) {
+    Boolean b = areColumnsUniqueWhenNullsFiltered(mq, rel, columnRefs);
     if (b == null) {
       return false;
     }
@@ -476,9 +463,9 @@ public class RelMdUtil {
       RexBuilder rexBuilder,
       RexNode pred1,
       RexNode pred2) {
-    List<RexNode> list1 = RelOptUtil.conjunctions(pred1);
-    List<RexNode> list2 = RelOptUtil.conjunctions(pred2);
-    List<RexNode> minusList = new ArrayList<>();
+    final List<RexNode> list1 = RelOptUtil.conjunctions(pred1);
+    final List<RexNode> list2 = RelOptUtil.conjunctions(pred2);
+    final List<RexNode> minusList = new ArrayList<>();
 
     for (RexNode rex1 : list1) {
       boolean add = true;
@@ -550,14 +537,15 @@ public class RelMdUtil {
 
   /**
    * Computes the cardinality of a particular expression from the projection
-   * list
+   * list.
    *
    * @param rel  RelNode corresponding to the project
    * @param expr projection expression
    * @return cardinality
    */
-  public static Double cardOfProjExpr(Project rel, RexNode expr) {
-    return expr.accept(new CardOfProjExpr(rel));
+  public static Double cardOfProjExpr(RelMetadataQuery mq, Project rel,
+      RexNode expr) {
+    return expr.accept(new CardOfProjExpr(mq, rel));
   }
 
   /**
@@ -567,9 +555,8 @@ public class RelMdUtil {
    * @param groupKey keys to compute the population for
    * @return computed population size
    */
-  public static Double getJoinPopulationSize(
-      RelNode joinRel,
-      ImmutableBitSet groupKey) {
+  public static Double getJoinPopulationSize(RelMetadataQuery mq,
+      RelNode joinRel, ImmutableBitSet groupKey) {
     ImmutableBitSet.Builder leftMask = ImmutableBitSet.builder();
     ImmutableBitSet.Builder rightMask = ImmutableBitSet.builder();
     RelNode left = joinRel.getInputs().get(0);
@@ -577,23 +564,14 @@ public class RelMdUtil {
 
     // separate the mask into masks for the left and right
     RelMdUtil.setLeftRightBitmaps(
-        groupKey,
-        leftMask,
-        rightMask,
-        left.getRowType().getFieldCount());
+        groupKey, leftMask, rightMask, left.getRowType().getFieldCount());
 
     Double population =
         NumberUtil.multiply(
-            RelMetadataQuery.getPopulationSize(
-                left,
-                leftMask.build()),
-            RelMetadataQuery.getPopulationSize(
-                right,
-                rightMask.build()));
+            mq.getPopulationSize(left, leftMask.build()),
+            mq.getPopulationSize(right, rightMask.build()));
 
-    return RelMdUtil.numDistinctVals(
-        population,
-        RelMetadataQuery.getRowCount(joinRel));
+    return numDistinctVals(population, mq.getRowCount(joinRel));
   }
 
   /**
@@ -608,12 +586,9 @@ public class RelMdUtil {
    *                  otherwise use <code>left NDV * right NDV</code>.
    * @return number of distinct rows
    */
-  public static Double getJoinDistinctRowCount(
-      RelNode joinRel,
-      JoinRelType joinType,
-      ImmutableBitSet groupKey,
-      RexNode predicate,
-      boolean useMaxNdv) {
+  public static Double getJoinDistinctRowCount(RelMetadataQuery mq,
+      RelNode joinRel, JoinRelType joinType, ImmutableBitSet groupKey,
+      RexNode predicate, boolean useMaxNdv) {
     Double distRowCount;
     ImmutableBitSet.Builder leftMask = ImmutableBitSet.builder();
     ImmutableBitSet.Builder rightMask = ImmutableBitSet.builder();
@@ -630,10 +605,10 @@ public class RelMdUtil {
     RexNode leftPred = null;
     RexNode rightPred = null;
     if (predicate != null) {
-      List<RexNode> leftFilters = new ArrayList<>();
-      List<RexNode> rightFilters = new ArrayList<>();
-      List<RexNode> joinFilters = new ArrayList<>();
-      List<RexNode> predList = RelOptUtil.conjunctions(predicate);
+      final List<RexNode> leftFilters = new ArrayList<>();
+      final List<RexNode> rightFilters = new ArrayList<>();
+      final List<RexNode> joinFilters = new ArrayList<>();
+      final List<RexNode> predList = RelOptUtil.conjunctions(predicate);
 
       RelOptUtil.classifyFilters(
           joinRel,
@@ -655,35 +630,35 @@ public class RelMdUtil {
 
     if (useMaxNdv) {
       distRowCount = Math.max(
-          RelMetadataQuery.getDistinctRowCount(left, leftMask.build(),
-              leftPred),
-          RelMetadataQuery.getDistinctRowCount(right, rightMask.build(),
-              rightPred));
+          mq.getDistinctRowCount(left, leftMask.build(), leftPred),
+          mq.getDistinctRowCount(right, rightMask.build(), rightPred));
     } else {
       distRowCount =
         NumberUtil.multiply(
-            RelMetadataQuery.getDistinctRowCount(
-                left,
-                leftMask.build(),
-                leftPred),
-            RelMetadataQuery.getDistinctRowCount(
-                right,
-                rightMask.build(),
-                rightPred));
+            mq.getDistinctRowCount(left, leftMask.build(), leftPred),
+            mq.getDistinctRowCount(right, rightMask.build(), rightPred));
     }
 
-    return RelMdUtil.numDistinctVals(
-        distRowCount,
-        RelMetadataQuery.getRowCount(joinRel));
+    return RelMdUtil.numDistinctVals(distRowCount, mq.getRowCount(joinRel));
+  }
+
+  /** Returns an estimate of the number of rows returned by a {@link Union}
+   * (before duplicates are eliminated). */
+  public static double getUnionAllRowCount(RelMetadataQuery mq, Union rel) {
+    double rowCount = 0;
+    for (RelNode input : rel.getInputs()) {
+      rowCount += mq.getRowCount(input);
+    }
+    return rowCount;
   }
 
   /** Returns an estimate of the number of rows returned by a {@link Minus}. */
-  public static double getMinusRowCount(Minus minus) {
+  public static double getMinusRowCount(RelMetadataQuery mq, Minus minus) {
     // REVIEW jvs 30-May-2005:  I just pulled this out of a hat.
     final List<RelNode> inputs = minus.getInputs();
-    double dRows = RelMetadataQuery.getRowCount(inputs.get(0));
+    double dRows = mq.getRowCount(inputs.get(0));
     for (int i = 1; i < inputs.size(); i++) {
-      dRows -= 0.5 * RelMetadataQuery.getRowCount(inputs.get(i));
+      dRows -= 0.5 * mq.getRowCount(inputs.get(i));
     }
     if (dRows < 0) {
       dRows = 0;
@@ -692,16 +667,17 @@ public class RelMdUtil {
   }
 
   /** Returns an estimate of the number of rows returned by a {@link Join}. */
-  public static Double getJoinRowCount(Join join, RexNode condition) {
+  public static Double getJoinRowCount(RelMetadataQuery mq, Join join,
+      RexNode condition) {
     // Row count estimates of 0 will be rounded up to 1.
     // So, use maxRowCount where the product is very small.
-    final Double left = RelMetadataQuery.getRowCount(join.getLeft());
-    final Double right = RelMetadataQuery.getRowCount(join.getRight());
+    final Double left = mq.getRowCount(join.getLeft());
+    final Double right = mq.getRowCount(join.getRight());
     if (left == null || right == null) {
       return null;
     }
     if (left <= 1D || right <= 1D) {
-      Double max = RelMetadataQuery.getMaxRowCount(join);
+      Double max = mq.getMaxRowCount(join);
       if (max != null && max <= 1D) {
         return max;
       }
@@ -709,30 +685,51 @@ public class RelMdUtil {
     double product = left * right;
 
     // TODO:  correlation factor
-    return product * RelMetadataQuery.getSelectivity(join, condition);
+    return product * mq.getSelectivity(join, condition);
   }
 
   /** Returns an estimate of the number of rows returned by a
    * {@link SemiJoin}. */
-  public static Double getSemiJoinRowCount(RelNode left, RelNode right,
-      JoinRelType joinType, RexNode condition) {
+  public static Double getSemiJoinRowCount(RelMetadataQuery mq, RelNode left,
+      RelNode right, JoinRelType joinType, RexNode condition) {
     // TODO:  correlation factor
-    final Double leftCount = RelMetadataQuery.getRowCount(left);
+    final Double leftCount = mq.getRowCount(left);
     if (leftCount == null) {
       return null;
     }
     return leftCount * RexUtil.getSelectivity(condition);
   }
 
+  public static double estimateFilteredRows(RelNode child, RexProgram program,
+      RelMetadataQuery mq) {
+    // convert the program's RexLocalRef condition to an expanded RexNode
+    RexLocalRef programCondition = program.getCondition();
+    RexNode condition;
+    if (programCondition == null) {
+      condition = null;
+    } else {
+      condition = program.expandLocalRef(programCondition);
+    }
+    return estimateFilteredRows(child, condition, mq);
+  }
+
+  public static double estimateFilteredRows(RelNode child, RexNode condition,
+      RelMetadataQuery mq) {
+    return mq.getRowCount(child)
+        * mq.getSelectivity(child, condition);
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /** Visitor that walks over a scalar expression and computes the
    * cardinality of its result. */
   private static class CardOfProjExpr extends RexVisitorImpl<Double> {
+    private final RelMetadataQuery mq;
     private Project rel;
 
-    public CardOfProjExpr(Project rel) {
+    public CardOfProjExpr(RelMetadataQuery mq, Project rel) {
       super(true);
+      this.mq = mq;
       this.rel = rel;
     }
 
@@ -740,57 +737,45 @@ public class RelMdUtil {
       int index = var.getIndex();
       ImmutableBitSet col = ImmutableBitSet.of(index);
       Double distinctRowCount =
-          RelMetadataQuery.getDistinctRowCount(
-              rel.getInput(),
-              col,
-              null);
+          mq.getDistinctRowCount(rel.getInput(), col, null);
       if (distinctRowCount == null) {
         return null;
       } else {
-        return RelMdUtil.numDistinctVals(
-            distinctRowCount,
-            RelMetadataQuery.getRowCount(rel));
+        return numDistinctVals(distinctRowCount, mq.getRowCount(rel));
       }
     }
 
     public Double visitLiteral(RexLiteral literal) {
-      return RelMdUtil.numDistinctVals(
-          1.0,
-          RelMetadataQuery.getRowCount(rel));
+      return numDistinctVals(1.0, mq.getRowCount(rel));
     }
 
     public Double visitCall(RexCall call) {
       Double distinctRowCount;
-      Double rowCount = RelMetadataQuery.getRowCount(rel);
+      Double rowCount = mq.getRowCount(rel);
       if (call.isA(SqlKind.MINUS_PREFIX)) {
-        distinctRowCount =
-            cardOfProjExpr(rel, call.getOperands().get(0));
+        distinctRowCount = cardOfProjExpr(mq, rel, call.getOperands().get(0));
       } else if (call.isA(ImmutableList.of(SqlKind.PLUS, SqlKind.MINUS))) {
-        Double card0 = cardOfProjExpr(rel, call.getOperands().get(0));
+        Double card0 = cardOfProjExpr(mq, rel, call.getOperands().get(0));
         if (card0 == null) {
           return null;
         }
-        Double card1 = cardOfProjExpr(rel, call.getOperands().get(1));
+        Double card1 = cardOfProjExpr(mq, rel, call.getOperands().get(1));
         if (card1 == null) {
           return null;
         }
         distinctRowCount = Math.max(card0, card1);
-      } else if (call.isA(
-          ImmutableList.of(SqlKind.TIMES, SqlKind.DIVIDE))) {
+      } else if (call.isA(ImmutableList.of(SqlKind.TIMES, SqlKind.DIVIDE))) {
         distinctRowCount =
             NumberUtil.multiply(
-                cardOfProjExpr(rel, call.getOperands().get(0)),
-                cardOfProjExpr(rel, call.getOperands().get(1)));
+                cardOfProjExpr(mq, rel, call.getOperands().get(0)),
+                cardOfProjExpr(mq, rel, call.getOperands().get(1)));
 
         // TODO zfong 6/21/06 - Broadbase has code to handle date
         // functions like year, month, day; E.g., cardinality of Month()
         // is 12
       } else {
         if (call.getOperands().size() == 1) {
-          distinctRowCount =
-              cardOfProjExpr(
-                  rel,
-                  call.getOperands().get(0));
+          distinctRowCount = cardOfProjExpr(mq, rel, call.getOperands().get(0));
         } else {
           distinctRowCount = rowCount / 10;
         }
@@ -805,13 +790,11 @@ public class RelMdUtil {
    *
    * <p>If this is the case, it is safe to push down a
    * {@link org.apache.calcite.rel.core.Sort} with limit and optional offset. */
-  public static boolean checkInputForCollationAndLimit(RelNode input,
-      RelCollation collation, RexNode offset, RexNode fetch) {
+  public static boolean checkInputForCollationAndLimit(RelMetadataQuery mq,
+      RelNode input, RelCollation collation, RexNode offset, RexNode fetch) {
     // Check if the input is already sorted
-    ImmutableList<RelCollation> inputCollations =
-        RelMetadataQuery.collations(input);
     boolean alreadySorted = false;
-    for (RelCollation inputCollation : inputCollations) {
+    for (RelCollation inputCollation : mq.collations(input)) {
       if (inputCollation.satisfies(collation)) {
         alreadySorted = true;
         break;
@@ -819,7 +802,7 @@ public class RelMdUtil {
     }
     // Check if we are not reducing the number of tuples
     boolean alreadySmaller = true;
-    final Double rowCount = RelMetadataQuery.getMaxRowCount(input);
+    final Double rowCount = mq.getMaxRowCount(input);
     if (rowCount != null && fetch != null) {
       final int offsetVal = offset == null ? 0 : RexLiteral.intValue(offset);
       final int limit = RexLiteral.intValue(fetch);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java
index 0e1bf4f..a2fd272 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataProvider.java
@@ -18,8 +18,6 @@ package org.apache.calcite.rel.metadata;
 
 import org.apache.calcite.rel.RelNode;
 
-import com.google.common.base.Function;
-
 /**
  * RelMetadataProvider defines an interface for obtaining metadata about
  * relational expressions. This interface is weakly-typed and is not intended to
@@ -56,9 +54,9 @@ public interface RelMetadataProvider {
    * @return Function that will field a metadata instance; or null if this
    *     provider cannot supply metadata of this type
    */
-  Function<RelNode, Metadata> apply(
-      Class<? extends RelNode> relClass,
-      Class<? extends Metadata> metadataClass);
+  <M extends Metadata> UnboundMetadata<M>
+  apply(Class<? extends RelNode> relClass,
+      Class<? extends M> metadataClass);
 }
 
 // End RelMetadataProvider.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
index 44d0724..f4eb748 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
@@ -30,6 +30,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -64,14 +65,26 @@ import java.util.Set;
  * custom providers for the standard queries in order to handle additional
  * relational expressions (either logical or physical). In either case, the
  * process is the same: write a reflective provider and chain it on to an
- * instance of {@link DefaultRelMetadataProvider}, prepending it to the default
+ * instance of {@link DefaultRelMetadataProvider}, pre-pending it to the default
  * providers. Then supply that instance to the planner via the appropriate
  * plugin mechanism.
  */
 public abstract class RelMetadataQuery {
+  /** Set of active metadata queries. */
+  public final Set<List> set = new HashSet<>();
+
   //~ Methods ----------------------------------------------------------------
 
   /**
+   * Returns an instance of RelMetadataQuery. It ensures that cycles do not
+   * occur while computing metadata.
+   */
+  public static RelMetadataQuery instance() {
+    return new RelMetadataQuery() {
+    };
+  }
+
+  /**
    * Returns the
    * {@link BuiltInMetadata.RowCount#getRowCount()}
    * statistic.
@@ -80,9 +93,9 @@ public abstract class RelMetadataQuery {
    * @return estimated row count, or null if no reliable estimate can be
    * determined
    */
-  public static Double getRowCount(RelNode rel) {
+  public Double getRowCount(RelNode rel) {
     final BuiltInMetadata.RowCount metadata =
-        rel.metadata(BuiltInMetadata.RowCount.class);
+        rel.metadata(BuiltInMetadata.RowCount.class, this);
     Double result = metadata.getRowCount();
     return validateResult(result);
   }
@@ -95,9 +108,9 @@ public abstract class RelMetadataQuery {
    * @param rel the relational expression
    * @return max row count
    */
-  public static Double getMaxRowCount(RelNode rel) {
+  public Double getMaxRowCount(RelNode rel) {
     final BuiltInMetadata.MaxRowCount metadata =
-        rel.metadata(BuiltInMetadata.MaxRowCount.class);
+        rel.metadata(BuiltInMetadata.MaxRowCount.class, this);
     return metadata.getMaxRowCount();
   }
 
@@ -109,9 +122,9 @@ public abstract class RelMetadataQuery {
    * @param rel the relational expression
    * @return estimated cost, or null if no reliable estimate can be determined
    */
-  public static RelOptCost getCumulativeCost(RelNode rel) {
+  public RelOptCost getCumulativeCost(RelNode rel) {
     final BuiltInMetadata.CumulativeCost metadata =
-        rel.metadata(BuiltInMetadata.CumulativeCost.class);
+        rel.metadata(BuiltInMetadata.CumulativeCost.class, this);
     return metadata.getCumulativeCost();
   }
 
@@ -123,9 +136,9 @@ public abstract class RelMetadataQuery {
    * @param rel the relational expression
    * @return estimated cost, or null if no reliable estimate can be determined
    */
-  public static RelOptCost getNonCumulativeCost(RelNode rel) {
+  public RelOptCost getNonCumulativeCost(RelNode rel) {
     final BuiltInMetadata.NonCumulativeCost metadata =
-        rel.metadata(BuiltInMetadata.NonCumulativeCost.class);
+        rel.metadata(BuiltInMetadata.NonCumulativeCost.class, this);
     return metadata.getNonCumulativeCost();
   }
 
@@ -138,9 +151,9 @@ public abstract class RelMetadataQuery {
    * @return estimated percentage (between 0.0 and 1.0), or null if no
    * reliable estimate can be determined
    */
-  public static Double getPercentageOriginalRows(RelNode rel) {
+  public Double getPercentageOriginalRows(RelNode rel) {
     final BuiltInMetadata.PercentageOriginalRows metadata =
-        rel.metadata(BuiltInMetadata.PercentageOriginalRows.class);
+        rel.metadata(BuiltInMetadata.PercentageOriginalRows.class, this);
     Double result = metadata.getPercentageOriginalRows();
     assert isPercentage(result, true);
     return result;
@@ -157,9 +170,9 @@ public abstract class RelMetadataQuery {
    * determined (whereas empty set indicates definitely no origin columns at
    * all)
    */
-  public static Set<RelColumnOrigin> getColumnOrigins(RelNode rel, int column) {
+  public Set<RelColumnOrigin> getColumnOrigins(RelNode rel, int column) {
     final BuiltInMetadata.ColumnOrigin metadata =
-        rel.metadata(BuiltInMetadata.ColumnOrigin.class);
+        rel.metadata(BuiltInMetadata.ColumnOrigin.class, this);
     return metadata.getColumnOrigins(column);
   }
 
@@ -176,7 +189,7 @@ public abstract class RelMetadataQuery {
    * @return the origin of a column provided it's a simple column; otherwise,
    * returns null
    */
-  public static RelColumnOrigin getColumnOrigin(RelNode rel, int column) {
+  public RelColumnOrigin getColumnOrigin(RelNode rel, int column) {
     final Set<RelColumnOrigin> origins = getColumnOrigins(rel, column);
     if (origins == null || origins.size() != 1) {
       return null;
@@ -193,12 +206,11 @@ public abstract class RelMetadataQuery {
    *
    * @return the table, if the RelNode is a simple table; otherwise null
    */
-  public static RelOptTable getTableOrigin(RelNode rel) {
+  public RelOptTable getTableOrigin(RelNode rel) {
     // Determine the simple origin of the first column in the
     // RelNode.  If it's simple, then that means that the underlying
     // table is also simple, even if the column itself is derived.
-    final Set<RelColumnOrigin> colOrigins =
-        getColumnOrigins(rel, 0);
+    final Set<RelColumnOrigin> colOrigins = getColumnOrigins(rel, 0);
     if (colOrigins == null || colOrigins.size() == 0) {
       return null;
     }
@@ -212,13 +224,13 @@ public abstract class RelMetadataQuery {
    *
    * @param rel       the relational expression
    * @param predicate predicate whose selectivity is to be estimated against
-   *                  rel's output
+   *                  {@code rel}'s output
    * @return estimated selectivity (between 0.0 and 1.0), or null if no
    * reliable estimate can be determined
    */
-  public static Double getSelectivity(RelNode rel, RexNode predicate) {
+  public Double getSelectivity(RelNode rel, RexNode predicate) {
     final BuiltInMetadata.Selectivity metadata =
-        rel.metadata(BuiltInMetadata.Selectivity.class);
+        rel.metadata(BuiltInMetadata.Selectivity.class, this);
     Double result = metadata.getSelectivity(predicate);
     assert isPercentage(result, true);
     return result;
@@ -233,9 +245,9 @@ public abstract class RelMetadataQuery {
    * @return set of keys, or null if this information cannot be determined
    * (whereas empty set indicates definitely no keys at all)
    */
-  public static Set<ImmutableBitSet> getUniqueKeys(RelNode rel) {
+  public Set<ImmutableBitSet> getUniqueKeys(RelNode rel) {
     final BuiltInMetadata.UniqueKeys metadata =
-        rel.metadata(BuiltInMetadata.UniqueKeys.class);
+        rel.metadata(BuiltInMetadata.UniqueKeys.class, this);
     return metadata.getUniqueKeys(false);
   }
 
@@ -247,13 +259,14 @@ public abstract class RelMetadataQuery {
    * @param rel         the relational expression
    * @param ignoreNulls if true, ignore null values when determining
    *                    whether the keys are unique
+   *
    * @return set of keys, or null if this information cannot be determined
    * (whereas empty set indicates definitely no keys at all)
    */
-  public static Set<ImmutableBitSet> getUniqueKeys(RelNode rel,
+  public Set<ImmutableBitSet> getUniqueKeys(RelNode rel,
       boolean ignoreNulls) {
     final BuiltInMetadata.UniqueKeys metadata =
-        rel.metadata(BuiltInMetadata.UniqueKeys.class);
+        rel.metadata(BuiltInMetadata.UniqueKeys.class, this);
     return metadata.getUniqueKeys(ignoreNulls);
   }
 
@@ -264,12 +277,13 @@ public abstract class RelMetadataQuery {
    * statistic over all columns.
    *
    * @param rel     the relational expression
+   *
    * @return true or false depending on whether the rows are unique, or
    * null if not enough information is available to make that determination
    */
-  public static Boolean areRowsUnique(RelNode rel) {
+  public Boolean areRowsUnique(RelNode rel) {
     final BuiltInMetadata.ColumnUniqueness metadata =
-        rel.metadata(BuiltInMetadata.ColumnUniqueness.class);
+        rel.metadata(BuiltInMetadata.ColumnUniqueness.class, this);
     final ImmutableBitSet columns =
         ImmutableBitSet.range(rel.getRowType().getFieldCount());
     return metadata.areColumnsUnique(columns, false);
@@ -277,24 +291,25 @@ public abstract class RelMetadataQuery {
 
   /**
    * Returns the
-   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(org.apache.calcite.util.ImmutableBitSet, boolean)}
+   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(ImmutableBitSet, boolean)}
    * statistic.
    *
    * @param rel     the relational expression
    * @param columns column mask representing the subset of columns for which
    *                uniqueness will be determined
+   *
    * @return true or false depending on whether the columns are unique, or
    * null if not enough information is available to make that determination
    */
-  public static Boolean areColumnsUnique(RelNode rel, ImmutableBitSet columns) {
+  public Boolean areColumnsUnique(RelNode rel, ImmutableBitSet columns) {
     final BuiltInMetadata.ColumnUniqueness metadata =
-        rel.metadata(BuiltInMetadata.ColumnUniqueness.class);
+        rel.metadata(BuiltInMetadata.ColumnUniqueness.class, this);
     return metadata.areColumnsUnique(columns, false);
   }
 
   /**
    * Returns the
-   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(org.apache.calcite.util.ImmutableBitSet, boolean)}
+   * {@link BuiltInMetadata.ColumnUniqueness#areColumnsUnique(ImmutableBitSet, boolean)}
    * statistic.
    *
    * @param rel         the relational expression
@@ -305,46 +320,46 @@ public abstract class RelMetadataQuery {
    * @return true or false depending on whether the columns are unique, or
    * null if not enough information is available to make that determination
    */
-  public static Boolean areColumnsUnique(RelNode rel, ImmutableBitSet columns,
+  public Boolean areColumnsUnique(RelNode rel, ImmutableBitSet columns,
       boolean ignoreNulls) {
     final BuiltInMetadata.ColumnUniqueness metadata =
-        rel.metadata(BuiltInMetadata.ColumnUniqueness.class);
+        rel.metadata(BuiltInMetadata.ColumnUniqueness.class, this);
     return metadata.areColumnsUnique(columns, ignoreNulls);
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Collation#collations()}
+   * {@link BuiltInMetadata.Collation#collations()}
    * statistic.
    *
    * @param rel         the relational expression
    * @return List of sorted column combinations, or
    * null if not enough information is available to make that determination
    */
-  public static ImmutableList<RelCollation> collations(RelNode rel) {
+  public ImmutableList<RelCollation> collations(RelNode rel) {
     final BuiltInMetadata.Collation metadata =
-        rel.metadata(BuiltInMetadata.Collation.class);
+        rel.metadata(BuiltInMetadata.Collation.class, this);
     return metadata.collations();
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Distribution#distribution()}
+   * {@link BuiltInMetadata.Distribution#distribution()}
    * statistic.
    *
    * @param rel         the relational expression
    * @return List of sorted column combinations, or
    * null if not enough information is available to make that determination
    */
-  public static RelDistribution distribution(RelNode rel) {
+  public RelDistribution distribution(RelNode rel) {
     final BuiltInMetadata.Distribution metadata =
-        rel.metadata(BuiltInMetadata.Distribution.class);
+        rel.metadata(BuiltInMetadata.Distribution.class, this);
     return metadata.distribution();
   }
 
   /**
    * Returns the
-   * {@link BuiltInMetadata.PopulationSize#getPopulationSize(org.apache.calcite.util.ImmutableBitSet)}
+   * {@link BuiltInMetadata.PopulationSize#getPopulationSize(ImmutableBitSet)}
    * statistic.
    *
    * @param rel      the relational expression
@@ -354,31 +369,31 @@ public abstract class RelMetadataQuery {
    * estimate can be determined
    *
    */
-  public static Double getPopulationSize(RelNode rel,
+  public Double getPopulationSize(RelNode rel,
       ImmutableBitSet groupKey) {
     final BuiltInMetadata.PopulationSize metadata =
-        rel.metadata(BuiltInMetadata.PopulationSize.class);
+        rel.metadata(BuiltInMetadata.PopulationSize.class, this);
     Double result = metadata.getPopulationSize(groupKey);
     return validateResult(result);
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Size#averageRowSize()}
+   * {@link BuiltInMetadata.Size#averageRowSize()}
    * statistic.
    *
    * @param rel      the relational expression
    * @return average size of a row, in bytes, or null if not known
      */
-  public static Double getAverageRowSize(RelNode rel) {
+  public Double getAverageRowSize(RelNode rel) {
     final BuiltInMetadata.Size metadata =
-        rel.metadata(BuiltInMetadata.Size.class);
+        rel.metadata(BuiltInMetadata.Size.class, this);
     return metadata.averageRowSize();
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Size#averageColumnSizes()}
+   * {@link BuiltInMetadata.Size#averageColumnSizes()}
    * statistic.
    *
    * @param rel      the relational expression
@@ -386,17 +401,17 @@ public abstract class RelMetadataQuery {
    * value, in bytes. Each value or the entire list may be null if the
    * metadata is not available
    */
-  public static List<Double> getAverageColumnSizes(RelNode rel) {
+  public List<Double> getAverageColumnSizes(RelNode rel) {
     final BuiltInMetadata.Size metadata =
-        rel.metadata(BuiltInMetadata.Size.class);
+        rel.metadata(BuiltInMetadata.Size.class, this);
     return metadata.averageColumnSizes();
   }
 
   /** As {@link #getAverageColumnSizes(org.apache.calcite.rel.RelNode)} but
    * never returns a null list, only ever a list of nulls. */
-  public static List<Double> getAverageColumnSizesNotNull(RelNode rel) {
+  public List<Double> getAverageColumnSizesNotNull(RelNode rel) {
     final BuiltInMetadata.Size metadata =
-        rel.metadata(BuiltInMetadata.Size.class);
+        rel.metadata(BuiltInMetadata.Size.class, this);
     final List<Double> averageColumnSizes = metadata.averageColumnSizes();
     return averageColumnSizes == null
         ? Collections.<Double>nCopies(rel.getRowType().getFieldCount(), null)
@@ -405,7 +420,7 @@ public abstract class RelMetadataQuery {
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Parallelism#isPhaseTransition()}
+   * {@link BuiltInMetadata.Parallelism#isPhaseTransition()}
    * statistic.
    *
    * @param rel      the relational expression
@@ -413,29 +428,29 @@ public abstract class RelMetadataQuery {
    * expression belongs to a different process than its inputs, or null if not
    * known
    */
-  public static Boolean isPhaseTransition(RelNode rel) {
+  public Boolean isPhaseTransition(RelNode rel) {
     final BuiltInMetadata.Parallelism metadata =
-        rel.metadata(BuiltInMetadata.Parallelism.class);
+        rel.metadata(BuiltInMetadata.Parallelism.class, this);
     return metadata.isPhaseTransition();
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Parallelism#splitCount()}
+   * {@link BuiltInMetadata.Parallelism#splitCount()}
    * statistic.
    *
    * @param rel      the relational expression
    * @return the number of distinct splits of the data, or null if not known
    */
-  public static Integer splitCount(RelNode rel) {
+  public Integer splitCount(RelNode rel) {
     final BuiltInMetadata.Parallelism metadata =
-        rel.metadata(BuiltInMetadata.Parallelism.class);
+        rel.metadata(BuiltInMetadata.Parallelism.class, this);
     return metadata.splitCount();
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Memory#memory()}
+   * {@link BuiltInMetadata.Memory#memory()}
    * statistic.
    *
    * @param rel      the relational expression
@@ -443,15 +458,15 @@ public abstract class RelMetadataQuery {
    * operator implementing this relational expression, across all splits,
    * or null if not known
    */
-  public static Double memory(RelNode rel) {
+  public Double memory(RelNode rel) {
     final BuiltInMetadata.Memory metadata =
-        rel.metadata(BuiltInMetadata.Memory.class);
+        rel.metadata(BuiltInMetadata.Memory.class, this);
     return metadata.memory();
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Memory#cumulativeMemoryWithinPhase()}
+   * {@link BuiltInMetadata.Memory#cumulativeMemoryWithinPhase()}
    * statistic.
    *
    * @param rel      the relational expression
@@ -459,15 +474,15 @@ public abstract class RelMetadataQuery {
    * physical operator implementing this relational expression, and all other
    * operators within the same phase, across all splits, or null if not known
    */
-  public static Double cumulativeMemoryWithinPhase(RelNode rel) {
+  public Double cumulativeMemoryWithinPhase(RelNode rel) {
     final BuiltInMetadata.Memory metadata =
-        rel.metadata(BuiltInMetadata.Memory.class);
+        rel.metadata(BuiltInMetadata.Memory.class, this);
     return metadata.cumulativeMemoryWithinPhase();
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Memory#cumulativeMemoryWithinPhaseSplit()}
+   * {@link BuiltInMetadata.Memory#cumulativeMemoryWithinPhaseSplit()}
    * statistic.
    *
    * @param rel      the relational expression
@@ -475,15 +490,15 @@ public abstract class RelMetadataQuery {
    * the physical operator implementing this relational expression, and all
    * operators within the same phase, within each split, or null if not known
    */
-  public static Double cumulativeMemoryWithinPhaseSplit(RelNode rel) {
+  public Double cumulativeMemoryWithinPhaseSplit(RelNode rel) {
     final BuiltInMetadata.Memory metadata =
-        rel.metadata(BuiltInMetadata.Memory.class);
+        rel.metadata(BuiltInMetadata.Memory.class, this);
     return metadata.cumulativeMemoryWithinPhaseSplit();
   }
 
   /**
    * Returns the
-   * {@link BuiltInMetadata.DistinctRowCount#getDistinctRowCount(org.apache.calcite.util.ImmutableBitSet, org.apache.calcite.rex.RexNode)}
+   * {@link BuiltInMetadata.DistinctRowCount#getDistinctRowCount(ImmutableBitSet, RexNode)}
    * statistic.
    *
    * @param rel       the relational expression
@@ -492,27 +507,27 @@ public abstract class RelMetadataQuery {
    * @return distinct row count for groupKey, filtered by predicate, or null
    * if no reliable estimate can be determined
    */
-  public static Double getDistinctRowCount(
+  public Double getDistinctRowCount(
       RelNode rel,
       ImmutableBitSet groupKey,
       RexNode predicate) {
     final BuiltInMetadata.DistinctRowCount metadata =
-        rel.metadata(BuiltInMetadata.DistinctRowCount.class);
+        rel.metadata(BuiltInMetadata.DistinctRowCount.class, this);
     Double result = metadata.getDistinctRowCount(groupKey, predicate);
     return validateResult(result);
   }
 
   /**
    * Returns the
-   * {@link org.apache.calcite.rel.metadata.BuiltInMetadata.Predicates#getPredicates()}
+   * {@link BuiltInMetadata.Predicates#getPredicates()}
    * statistic.
    *
    * @param rel the relational expression
    * @return Predicates that can be pulled above this RelNode
    */
-  public static RelOptPredicateList getPulledUpPredicates(RelNode rel) {
+  public RelOptPredicateList getPulledUpPredicates(RelNode rel) {
     final BuiltInMetadata.Predicates metadata =
-        rel.metadata(BuiltInMetadata.Predicates.class);
+        rel.metadata(BuiltInMetadata.Predicates.class, this);
     return metadata.getPredicates();
   }
 
@@ -526,10 +541,10 @@ public abstract class RelMetadataQuery {
    * @return true for visible, false for invisible; if no metadata is available,
    * defaults to true
    */
-  public static boolean isVisibleInExplain(RelNode rel,
+  public boolean isVisibleInExplain(RelNode rel,
       SqlExplainLevel explainLevel) {
     final BuiltInMetadata.ExplainVisibility metadata =
-        rel.metadata(BuiltInMetadata.ExplainVisibility.class);
+        rel.metadata(BuiltInMetadata.ExplainVisibility.class, this);
     Boolean b = metadata.isVisibleInExplain(explainLevel);
     return b == null || b;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/metadata/UnboundMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/UnboundMetadata.java b/core/src/main/java/org/apache/calcite/rel/metadata/UnboundMetadata.java
new file mode 100644
index 0000000..ef8dc4a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/UnboundMetadata.java
@@ -0,0 +1,31 @@
+/*
+ * 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.rel.metadata;
+
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * Metadata that needs to be bound to a {@link RelNode} and
+ * {@link RelMetadataQuery} before it can be used.
+ *
+ * @param <M> Metadata type
+ */
+public interface UnboundMetadata<M extends Metadata> {
+  M bind(RelNode rel, RelMetadataQuery mq);
+}
+
+// End UnboundMetadata.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 52b5497..b2ebf84 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
@@ -72,8 +72,8 @@ public class AggregateFilterTransposeRule extends RelOptRule {
     final ImmutableBitSet newGroupSet =
         aggregate.getGroupSet().union(filterColumns);
     final RelNode input = filter.getInput();
-    final Boolean unique =
-        RelMetadataQuery.areColumnsUnique(input, newGroupSet);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final Boolean unique = mq.areColumnsUnique(input, newGroupSet);
     if (unique != null && unique) {
       // The input is already unique on the grouping columns, so there's little
       // advantage of aggregating again. More important, without this check,

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 016e45a..6978e04 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
@@ -38,7 +38,9 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlSplittableAggFunction;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.Mappings;
 
@@ -153,8 +155,9 @@ public class AggregateJoinTransposeRule extends RelOptRule {
 
     // Do the columns used by the join appear in the output of the aggregate?
     final ImmutableBitSet aggregateColumns = aggregate.getGroupSet();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final ImmutableBitSet keyColumns = keyColumns(aggregateColumns,
-        RelMetadataQuery.getPulledUpPredicates(join).pulledUpPredicates);
+        mq.getPulledUpPredicates(join).pulledUpPredicates);
     final ImmutableBitSet joinColumns =
         RelOptUtil.InputFinder.bits(join.getCondition());
     final boolean allColumnsInAggregate =
@@ -202,17 +205,18 @@ public class AggregateJoinTransposeRule extends RelOptRule {
         // any functions experiencing a cartesian product effect.
         //
         // But finding out whether the input is already unique requires a call
-        // to areColumnsUnique that currently (until [CALCITE-794] "Detect
-        // cycles when computing statistics" is fixed) places a heavy load on
+        // to areColumnsUnique that currently (until [CALCITE-1048] "Make
+        // metadata more robust" is fixed) places a heavy load on
         // the metadata system.
         //
         // So we choose to imagine the the input is already unique, which is
         // untrue but harmless.
         //
+        Util.discard(Bug.CALCITE_1048_FIXED);
         unique = true;
       } else {
         final Boolean unique0 =
-            RelMetadataQuery.areColumnsUnique(joinInput, belowAggregateKey);
+            mq.areColumnsUnique(joinInput, belowAggregateKey);
         unique = unique0 != null && unique0;
       }
       if (unique) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 a0ff130..bb7797e 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
@@ -109,8 +109,9 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
     }
 
     final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     final RelOptPredicateList predicates =
-        RelMetadataQuery.getPulledUpPredicates(aggregate.getInput());
+        mq.getPulledUpPredicates(aggregate.getInput());
     if (predicates == null) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
index aa74448..b98ba64 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
@@ -57,10 +57,11 @@ public class AggregateRemoveRule extends RelOptRule {
   public void onMatch(RelOptRuleCall call) {
     final LogicalAggregate aggregate = call.rel(0);
     final RelNode input = call.rel(1);
-    if (!aggregate.getAggCallList().isEmpty()
-        || aggregate.indicator
-        || !SqlFunctions.isTrue(
-            RelMetadataQuery.areColumnsUnique(input, aggregate.getGroupSet()))) {
+    if (!aggregate.getAggCallList().isEmpty() || aggregate.indicator) {
+      return;
+    }
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    if (!SqlFunctions.isTrue(mq.areColumnsUnique(input, aggregate.getGroupSet()))) {
       return;
     }
     // Distinct is "GROUP BY c1, c2" (where c1, c2 are a set of columns on

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
index 7464a0f..3945924 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 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.schema.Table;
 import org.apache.calcite.schema.impl.StarTable;
@@ -115,7 +116,8 @@ public class AggregateStarTableRule extends RelOptRule {
     final RelBuilder relBuilder = call.builder();
     final CalciteSchema.TableEntry tableEntry = pair.left;
     final TileKey tileKey = pair.right;
-    final double rowCount = aggregate.getRows();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final double rowCount = aggregate.estimateRowCount(mq);
     final Table aggregateTable = tableEntry.getTable();
     final RelDataType aggregateTableRowType =
         aggregateTable.getRowType(cluster.getTypeFactory());

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/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 2a2ef98..12300d9 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
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.fun.SqlCountAggFunction;
@@ -115,10 +116,10 @@ public class AggregateUnionTransposeRule extends RelOptRule {
     // create corresponding aggregates on top of each union child
     final RelBuilder relBuilder = call.builder();
     int transformCount = 0;
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     for (RelNode input : union.getInputs()) {
       boolean alreadyUnique =
-          RelMdUtil.areColumnsDefinitelyUnique(
-              input,
+          RelMdUtil.areColumnsDefinitelyUnique(mq, input,
               aggRel.getGroupSet());
 
       relBuilder.push(input);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java
index 1d375c6..0b020c7 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinPushTransitivePredicatesRule.java
@@ -59,7 +59,8 @@ public class JoinPushTransitivePredicatesRule extends RelOptRule {
 
   @Override public void onMatch(RelOptRuleCall call) {
     Join join = call.rel(0);
-    RelOptPredicateList preds = RelMetadataQuery.getPulledUpPredicates(join);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    RelOptPredicateList preds = mq.getPulledUpPredicates(join);
 
     if (preds.leftInferredPredicates.isEmpty()
         && preds.rightInferredPredicates.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
index a41e1aa..ee4eeaa 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
@@ -192,7 +192,7 @@ public class LoptMultiJoin {
     joinFilters =
         Lists.newArrayList(RelOptUtil.conjunctions(multiJoin.getJoinFilter()));
 
-    allJoinFilters = new ArrayList<RexNode>(joinFilters);
+    allJoinFilters = new ArrayList<>(joinFilters);
     List<RexNode> outerJoinFilters = multiJoin.getOuterJoinConditions();
     for (int i = 0; i < nJoinFactors; i++) {
       allJoinFilters.addAll(RelOptUtil.conjunctions(outerJoinFilters.get(i)));
@@ -235,8 +235,8 @@ public class LoptMultiJoin {
     joinRemovalFactors = new Integer[nJoinFactors];
     joinRemovalSemiJoins = new SemiJoin[nJoinFactors];
 
-    removableOuterJoinFactors = new HashSet<Integer>();
-    removableSelfJoinPairs = new HashMap<Integer, RemovableSelfJoin>();
+    removableOuterJoinFactors = new HashSet<>();
+    removableSelfJoinPairs = new HashMap<>();
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -695,15 +695,15 @@ public class LoptMultiJoin {
     // Compute a column mapping such that if a column from the right
     // factor is also referenced in the left factor, we will map the
     // right reference to the left to avoid redundant references.
-    Map<Integer, Integer> columnMapping = new HashMap<Integer, Integer>();
+    final Map<Integer, Integer> columnMapping = new HashMap<>();
 
     // First, locate the originating column for all simple column
     // references in the left factor.
-    RelNode left = getJoinFactor(leftFactor);
-    Map<Integer, Integer> leftFactorColMapping =
-        new HashMap<Integer, Integer>();
+    final RelNode left = getJoinFactor(leftFactor);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final Map<Integer, Integer> leftFactorColMapping = new HashMap<>();
     for (int i = 0; i < left.getRowType().getFieldCount(); i++) {
-      RelColumnOrigin colOrigin = RelMetadataQuery.getColumnOrigin(left, i);
+      final RelColumnOrigin colOrigin = mq.getColumnOrigin(left, i);
       if (colOrigin != null) {
         leftFactorColMapping.put(
             colOrigin.getOriginColumnOrdinal(),
@@ -717,8 +717,7 @@ public class LoptMultiJoin {
     // factor.
     RelNode right = getJoinFactor(rightFactor);
     for (int i = 0; i < right.getRowType().getFieldCount(); i++) {
-      final RelColumnOrigin colOrigin =
-          RelMetadataQuery.getColumnOrigin(right, i);
+      final RelColumnOrigin colOrigin = mq.getColumnOrigin(right, i);
       if (colOrigin == null) {
         continue;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
index 66ae671..e522ade 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
@@ -209,9 +209,9 @@ public class LoptOptimizeJoinRule extends RelOptRule {
         // part of an equality join condition, nulls are filtered out
         // by the join.  So, it's ok if there are nulls in the join
         // keys.
-        if (RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
-            multiJoin.getJoinFactor(factIdx),
-            joinKeys)) {
+        final RelMetadataQuery mq = RelMetadataQuery.instance();
+        if (RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(mq,
+            multiJoin.getJoinFactor(factIdx), joinKeys)) {
           multiJoin.addRemovableOuterJoinFactor(factIdx);
 
           // Since we are no longer joining this factor,
@@ -319,7 +319,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     // the appropriate join condition between the two factors that will
     // allow the join to be removed.
     for (Integer factor1 : selfJoinPairs.keySet()) {
-      int factor2 = selfJoinPairs.get(factor1);
+      final int factor2 = selfJoinPairs.get(factor1);
       final List<RexNode> selfJoinFilters = new ArrayList<>();
       for (RexNode filter : multiJoin.getJoinFilters()) {
         ImmutableBitSet joinFactors =
@@ -361,13 +361,14 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     if (multiJoin.getMultiJoinRel().isFullOuterJoin()) {
       return returnList;
     }
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     for (int factIdx = 0; factIdx < multiJoin.getNumJoinFactors(); factIdx++) {
       if (multiJoin.isNullGenerating(factIdx)
           || (multiJoin.getJoinRemovalFactor(factIdx) != null)) {
         continue;
       }
       final RelNode rel = multiJoin.getJoinFactor(factIdx);
-      final RelOptTable table = RelMetadataQuery.getTableOrigin(rel);
+      final RelOptTable table = mq.getTableOrigin(rel);
       if (table != null) {
         returnList.put(factIdx, table);
       }
@@ -421,7 +422,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
                 rightRel.getRowType().getFieldList(),
                 adjustments));
 
-    return areSelfJoinKeysUnique(leftRel, rightRel, joinFilters);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    return areSelfJoinKeysUnique(mq, leftRel, rightRel, joinFilters);
   }
 
   /**
@@ -439,7 +441,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       RelOptRuleCall call) {
     final List<RelNode> plans = new ArrayList<>();
 
-    List<String> fieldNames =
+    final List<String> fieldNames =
         multiJoin.getMultiJoinRel().getRowType().getFieldNames();
 
     // generate the N join orderings
@@ -603,10 +605,9 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     if (joinKeys.isEmpty()) {
       return null;
     } else {
-      return RelMetadataQuery.getDistinctRowCount(
-          semiJoinOpt.getChosenSemiJoin(factor),
-          joinKeys.build(),
-          null);
+      final RelMetadataQuery mq = semiJoinOpt.mq;
+      return mq.getDistinctRowCount(semiJoinOpt.getChosenSemiJoin(factor),
+          joinKeys.build(), null);
     }
   }
 
@@ -671,9 +672,9 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       LoptSemiJoinOptimizer semiJoinOpt,
       int firstFactor) {
     LoptJoinTree joinTree = null;
-    int nJoinFactors = multiJoin.getNumJoinFactors();
-    BitSet factorsToAdd = BitSets.range(0, nJoinFactors);
-    BitSet factorsAdded = new BitSet(nJoinFactors);
+    final int nJoinFactors = multiJoin.getNumJoinFactors();
+    final BitSet factorsToAdd = BitSets.range(0, nJoinFactors);
+    final BitSet factorsAdded = new BitSet(nJoinFactors);
     final List<RexNode> filtersToAdd =
         new ArrayList<>(multiJoin.getJoinFilters());
 
@@ -867,6 +868,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       BitSet factorsNeeded,
       List<RexNode> filtersToAdd,
       boolean selfJoin) {
+    final RelMetadataQuery mq = semiJoinOpt.mq;
+
     // if the factor corresponds to the null generating factor in an outer
     // join that can be removed, then create a replacement join
     if (multiJoin.isRemovableOuterJoinFactor(factorToAdd)) {
@@ -935,11 +938,10 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     RelOptCost costPushDown = null;
     RelOptCost costTop = null;
     if (pushDownTree != null) {
-      costPushDown =
-          RelMetadataQuery.getCumulativeCost(pushDownTree.getJoinTree());
+      costPushDown = mq.getCumulativeCost(pushDownTree.getJoinTree());
     }
     if (topTree != null) {
-      costTop = RelMetadataQuery.getCumulativeCost(topTree.getJoinTree());
+      costTop = mq.getCumulativeCost(topTree.getJoinTree());
     }
 
     if (pushDownTree == null) {
@@ -1337,7 +1339,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       int factorAdded,
       List<Integer> origJoinOrder,
       List<RelDataTypeField> origFields) {
-    List<Integer> newJoinOrder = new ArrayList<>();
+    final List<Integer> newJoinOrder = new ArrayList<>();
     left.getTreeOrder(newJoinOrder);
     right.getTreeOrder(newJoinOrder);
 
@@ -1858,8 +1860,9 @@ public class LoptOptimizeJoinRule extends RelOptRule {
           ((LoptJoinTree.Leaf) left.getFactorTree()).getId());
     }
 
-    Double leftRowCount = RelMetadataQuery.getRowCount(left.getJoinTree());
-    Double rightRowCount = RelMetadataQuery.getRowCount(right.getJoinTree());
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final Double leftRowCount = mq.getRowCount(left.getJoinTree());
+    final Double rightRowCount = mq.getRowCount(right.getJoinTree());
 
     // The left side is smaller than the right if it has fewer rows,
     // or if it has the same number of rows as the right (excluding
@@ -1990,11 +1993,12 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     }
 
     // Make sure the join is between the same simple factor
-    final RelOptTable leftTable = RelMetadataQuery.getTableOrigin(left);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final RelOptTable leftTable = mq.getTableOrigin(left);
     if (leftTable == null) {
       return false;
     }
-    final RelOptTable rightTable = RelMetadataQuery.getTableOrigin(right);
+    final RelOptTable rightTable = mq.getTableOrigin(right);
     if (rightTable == null) {
       return false;
     }
@@ -2003,35 +2007,34 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     }
 
     // Determine if the join keys are identical and unique
-    return areSelfJoinKeysUnique(left, right, joinRel.getCondition());
+    return areSelfJoinKeysUnique(mq, left, right, joinRel.getCondition());
   }
 
   /**
    * Determines if the equality portion of a self-join condition is between
    * identical keys that are unique.
    *
+   * @param mq Metadata query
    * @param leftRel left side of the join
    * @param rightRel right side of the join
    * @param joinFilters the join condition
    *
    * @return true if the equality join keys are the same and unique
    */
-  private static boolean areSelfJoinKeysUnique(
-      RelNode leftRel,
-      RelNode rightRel,
-      RexNode joinFilters) {
+  private static boolean areSelfJoinKeysUnique(RelMetadataQuery mq,
+      RelNode leftRel, RelNode rightRel, RexNode joinFilters) {
     final JoinInfo joinInfo = JoinInfo.of(leftRel, rightRel, joinFilters);
 
     // Make sure each key on the left maps to the same simple column as the
     // corresponding key on the right
     for (IntPair pair : joinInfo.pairs()) {
       final RelColumnOrigin leftOrigin =
-          RelMetadataQuery.getColumnOrigin(leftRel, pair.source);
+          mq.getColumnOrigin(leftRel, pair.source);
       if (leftOrigin == null) {
         return false;
       }
       final RelColumnOrigin rightOrigin =
-          RelMetadataQuery.getColumnOrigin(rightRel, pair.target);
+          mq.getColumnOrigin(rightRel, pair.target);
       if (rightOrigin == null) {
         return false;
       }
@@ -2045,7 +2048,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     // are unique.  When removing self-joins, if needed, we'll later add an
     // IS NOT NULL filter on the join keys that are nullable.  Therefore,
     // it's ok if there are nulls in the unique key.
-    return RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(leftRel,
+    return RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(mq, leftRel,
         joinInfo.leftSet());
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
index 080cb9c..308091e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
@@ -58,10 +58,11 @@ public class LoptSemiJoinOptimizer {
 
   //~ Instance fields --------------------------------------------------------
 
-  /**
-   * RexBuilder for constructing new RexNodes
-   */
-  private RexBuilder rexBuilder;
+  private final RexBuilder rexBuilder;
+
+  /** Not thread-safe. But should be OK, because an optimizer is only used
+   * from within one thread.*/
+  final RelMetadataQuery mq = RelMetadataQuery.instance();
 
   /**
    * Semijoins corresponding to each join factor, if they are going to be
@@ -406,7 +407,7 @@ public class LoptSemiJoinOptimizer {
     while (keyIter.hasNext()) {
       boolean removeKey = false;
       final RelColumnOrigin colOrigin =
-          RelMetadataQuery.getColumnOrigin(factRel, keyIter.next());
+          mq.getColumnOrigin(factRel, keyIter.next());
 
       // can't use the rid column as a semijoin key
       if ((colOrigin == null)
@@ -635,13 +636,13 @@ public class LoptSemiJoinOptimizer {
     // a middle ground based on testing that was done with a large
     // data set.
     final ImmutableBitSet dimCols = ImmutableBitSet.of(semiJoin.getRightKeys());
-    double selectivity =
-        RelMdUtil.computeSemiJoinSelectivity(factRel, dimRel, semiJoin);
+    final double selectivity =
+        RelMdUtil.computeSemiJoinSelectivity(mq, factRel, dimRel, semiJoin);
     if (selectivity > .5) {
       return 0;
     }
 
-    RelOptCost factCost = RelMetadataQuery.getCumulativeCost(factRel);
+    final RelOptCost factCost = mq.getCumulativeCost(factRel);
 
     // if not enough information, return a low score
     if (factCost == null) {
@@ -654,9 +655,8 @@ public class LoptSemiJoinOptimizer {
     // Additional savings if the dimension columns are unique.  We can
     // ignore nulls since they will be filtered out by the semijoin.
     boolean uniq =
-        RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
-            dimRel,
-            dimCols);
+        RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(mq,
+            dimRel, dimCols);
     if (uniq) {
       savings *= 2.0;
     }
@@ -664,9 +664,9 @@ public class LoptSemiJoinOptimizer {
     // compute the cost of doing an extra scan on the dimension table,
     // including the distinct sort on top of the scan; if the dimension
     // columns are already unique, no need to add on the dup removal cost
-    Double dimSortCost = RelMetadataQuery.getRowCount(dimRel);
-    Double dupRemCost = uniq ? 0 : dimSortCost;
-    RelOptCost dimCost = RelMetadataQuery.getCumulativeCost(dimRel);
+    final Double dimSortCost = mq.getRowCount(dimRel);
+    final Double dupRemCost = uniq ? 0 : dimSortCost;
+    final RelOptCost dimCost = mq.getCumulativeCost(dimRel);
     if ((dimSortCost == null)
         || (dupRemCost == null)
         || (dimCost == null)) {
@@ -708,9 +708,8 @@ public class LoptSemiJoinOptimizer {
     // Check if the semijoin keys corresponding to the dimension table
     // are unique.  The semijoin will filter out the nulls.
     final ImmutableBitSet dimKeys = ImmutableBitSet.of(semiJoin.getRightKeys());
-    RelNode dimRel = multiJoin.getJoinFactor(dimIdx);
-    if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
-        dimRel,
+    final RelNode dimRel = multiJoin.getJoinFactor(dimIdx);
+    if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(mq, dimRel,
         dimKeys)) {
       return;
     }
@@ -800,14 +799,14 @@ public class LoptSemiJoinOptimizer {
 
   //~ Inner Classes ----------------------------------------------------------
 
-  /** . */
+  /** Compares factors. */
   private class FactorCostComparator
       implements Comparator<Integer> {
     public int compare(Integer rel1Idx, Integer rel2Idx) {
       RelOptCost c1 =
-          RelMetadataQuery.getCumulativeCost(chosenSemiJoins[rel1Idx]);
+          mq.getCumulativeCost(chosenSemiJoins[rel1Idx]);
       RelOptCost c2 =
-          RelMetadataQuery.getCumulativeCost(chosenSemiJoins[rel2Idx]);
+          mq.getCumulativeCost(chosenSemiJoins[rel2Idx]);
 
       // nulls are arbitrarily sorted
       if ((c1 == null) || (c2 == null)) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
index 477b74a..b0b372d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoinOptimizeBushyRule.java
@@ -87,6 +87,7 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
     final MultiJoin multiJoinRel = call.rel(0);
     final RexBuilder rexBuilder = multiJoinRel.getCluster().getRexBuilder();
     final RelBuilder relBuilder = call.builder();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
 
     final LoptMultiJoin multiJoin = new LoptMultiJoin(multiJoinRel);
 
@@ -94,7 +95,7 @@ public class MultiJoinOptimizeBushyRule extends RelOptRule {
     int x = 0;
     for (int i = 0; i < multiJoin.getNumJoinFactors(); i++) {
       final RelNode rel = multiJoin.getJoinFactor(i);
-      double cost = RelMetadataQuery.getRowCount(rel);
+      double cost = mq.getRowCount(rel);
       vertexes.add(new LeafVertex(i, rel, cost, x));
       x += rel.getRowType().getFieldCount();
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 759748c..3fce8a6 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -140,8 +140,9 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
           Lists.newArrayList(filter.getCondition());
       RexNode newConditionExp;
       boolean reduced;
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
       final RelOptPredicateList predicates =
-          RelMetadataQuery.getPulledUpPredicates(filter.getInput());
+          mq.getPulledUpPredicates(filter.getInput());
       if (reduceExpressions(filter, expList, predicates)) {
         assert expList.size() == 1;
         newConditionExp = expList.get(0);
@@ -229,9 +230,10 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     }
 
     @Override public void onMatch(RelOptRuleCall call) {
-      Project project = call.rel(0);
+      final Project project = call.rel(0);
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
       final RelOptPredicateList predicates =
-          RelMetadataQuery.getPulledUpPredicates(project.getInput());
+          mq.getPulledUpPredicates(project.getInput());
       final List<RexNode> expList =
           Lists.newArrayList(project.getProjects());
       if (reduceExpressions(project, expList, predicates)) {
@@ -261,10 +263,11 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
       final Join join = call.rel(0);
       final List<RexNode> expList = Lists.newArrayList(join.getCondition());
       final int fieldCount = join.getLeft().getRowType().getFieldCount();
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
       final RelOptPredicateList leftPredicates =
-          RelMetadataQuery.getPulledUpPredicates(join.getLeft());
+          mq.getPulledUpPredicates(join.getLeft());
       final RelOptPredicateList rightPredicates =
-          RelMetadataQuery.getPulledUpPredicates(join.getRight());
+          mq.getPulledUpPredicates(join.getRight());
       final RelOptPredicateList predicates =
           leftPredicates.union(rightPredicates.shift(fieldCount));
       if (!reduceExpressions(join, expList, predicates)) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/SortJoinTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SortJoinTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SortJoinTransposeRule.java
index 66b0afb..2da9a35 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SortJoinTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SortJoinTransposeRule.java
@@ -29,6 +29,7 @@ import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 
 /**
@@ -99,11 +100,12 @@ public class SortJoinTransposeRule extends RelOptRule {
     // We create a new sort operator on the corresponding input
     final RelNode newLeftInput;
     final RelNode newRightInput;
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     if (join.getJoinType() == JoinRelType.LEFT) {
       // If the input is already sorted and we are not reducing the number of tuples,
       // we bail out
-      if (RelMdUtil.checkInputForCollationAndLimit(join.getLeft(), sort.getCollation(),
-          sort.offset, sort.fetch)) {
+      if (RelMdUtil.checkInputForCollationAndLimit(mq, join.getLeft(),
+          sort.getCollation(), sort.offset, sort.fetch)) {
         return;
       }
       newLeftInput = sort.copy(sort.getTraitSet(), join.getLeft(), sort.getCollation(),
@@ -116,8 +118,8 @@ public class SortJoinTransposeRule extends RelOptRule {
                   -join.getLeft().getRowType().getFieldCount()));
       // If the input is already sorted and we are not reducing the number of tuples,
       // we bail out
-      if (RelMdUtil.checkInputForCollationAndLimit(join.getRight(), rightCollation,
-          sort.offset, sort.fetch)) {
+      if (RelMdUtil.checkInputForCollationAndLimit(mq, join.getRight(),
+          rightCollation, sort.offset, sort.fetch)) {
         return;
       }
       newLeftInput = join.getLeft();

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/SortUnionTransposeRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SortUnionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SortUnionTransposeRule.java
index b195e3f..26d2106 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SortUnionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SortUnionTransposeRule.java
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.tools.RelBuilderFactory;
 
 import java.util.ArrayList;
@@ -93,9 +94,10 @@ public class SortUnionTransposeRule extends RelOptRule {
     // Thus we use 'ret' as a flag to identify if we have finished pushing the
     // sort past a union.
     boolean ret = true;
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
     for (RelNode input : union.getInputs()) {
-      if (!RelMdUtil.checkInputForCollationAndLimit(input, sort.getCollation(),
-          sort.offset, sort.fetch)) {
+      if (!RelMdUtil.checkInputForCollationAndLimit(mq, input,
+          sort.getCollation(), sort.offset, sort.fetch)) {
         ret = false;
         Sort branchSort = sort.copy(sort.getTraitSet(), input,
             sort.getCollation(), sort.offset, sort.fetch);

http://git-wip-us.apache.org/repos/asf/calcite/blob/cabdcf44/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
index c3daf49..56b362d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
@@ -148,7 +148,8 @@ public abstract class SubQueryRemoveRule extends RelOptRule {
     switch (e.getKind()) {
     case SCALAR_QUERY:
       builder.push(e.rel);
-      final Boolean unique = RelMetadataQuery.areColumnsUnique(builder.peek(),
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
+      final Boolean unique = mq.areColumnsUnique(builder.peek(),
           ImmutableBitSet.of());
       if (unique == null || !unique) {
         builder.aggregate(builder.groupKey(),


[21/50] [abbrv] calcite git commit: [CALCITE-816] Represent sub-query as a RexNode

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 553f275..764dde2 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -233,7 +233,9 @@ LogicalProject(DEPTNO=[$0], NAME=[$1], EMPSET=[$2])
 ]]>
         </Resource>
         <Resource name="sql">
-            <![CDATA[select *,         multiset(select * from emp where deptno=dept.deptno)                as empset      from dept]]>
+            <![CDATA[select *,
+  multiset(select * from emp where deptno=dept.deptno) as empset
+from dept]]>
         </Resource>
     </TestCase>
     <TestCase name="testExists">
@@ -1411,7 +1413,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testLateralDecorrelate">
         <Resource name="sql">
-            <![CDATA[select * from emp, LATERAL (select * from dept where emp.deptno=dept.deptno)]]>
+            <![CDATA[select * from emp,
+ LATERAL (select * from dept where emp.deptno=dept.deptno)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1459,7 +1462,8 @@ LogicalProject(D2=[$0], D3=[$1])
     </TestCase>
     <TestCase name="testNestedCorrelationsDecorrelated">
         <Resource name="sql">
-            <![CDATA[select * from (select 2+deptno d2, 3+deptno d3 from emp) e
+            <![CDATA[select *
+from (select 2+deptno d2, 3+deptno d3 from emp) e
  where exists (select 1 from (select deptno+1 d1 from dept) d
  where d1=e.d2 and exists (select 2 from (select deptno+4 d4, deptno+5 d5, deptno+6 d6 from dept)
  where d4=d.d1 and d5=d.d1 and d6=e.d3))]]>
@@ -1537,7 +1541,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testExistsCorrelatedDecorrelate">
         <Resource name="sql">
-            <![CDATA[select*from emp where exists (select 1 from dept where emp.deptno=dept.deptno)]]>
+            <![CDATA[select*from emp where exists (
+  select 1 from dept where emp.deptno=dept.deptno)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1586,16 +1591,20 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$9])
-    LogicalFilter(condition=[IS NOT NULL($9)])
-      LogicalCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{7}])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$10])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[CAST($9):INTEGER], $f1=[CAST($10):BOOLEAN])
+      LogicalJoin(condition=[=($7, $9)], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-        LogicalAggregate(group=[{}], agg#0=[MIN($0)])
-          LogicalProject($f0=[true])
-            LogicalSort(fetch=[1])
-              LogicalProject(EXPR$0=[1])
-                LogicalFilter(condition=[=($cor0.DEPTNO, $0)])
-                  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
+          LogicalProject(DEPTNO0=[$1], $f0=[$0])
+            LogicalProject($f0=[true], DEPTNO0=[$1])
+              LogicalSort(fetch=[1])
+                LogicalProject(EXPR$0=[1], DEPTNO0=[$2])
+                  LogicalJoin(condition=[=($2, $0)], joinType=[inner])
+                    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+                    LogicalAggregate(group=[{0}])
+                      LogicalProject(DEPTNO=[$7])
+                        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
@@ -2478,7 +2487,8 @@ LogicalAggregate(group=[{0}], EMPID=[MIN($1)])
     </TestCase>
     <TestCase name="testCorrelationScalarAggAndFilter">
         <Resource name="sql">
-            <![CDATA[SELECT e1.empno FROM emp e1, dept d1 where e1.deptno = d1.deptno
+            <![CDATA[SELECT e1.empno
+FROM emp e1, dept d1 where e1.deptno = d1.deptno
 and e1.deptno < 10 and d1.deptno < 15
 and e1.sal > (select avg(sal) from emp e2 where e1.empno = e2.empno)]]>
         </Resource>
@@ -2510,7 +2520,8 @@ LogicalProject(EMPNO=[$0])
     </TestCase>
     <TestCase name="testCorrelationExistsAndFilter">
         <Resource name="sql">
-            <![CDATA[SELECT e1.empno FROM emp e1, dept d1 where e1.deptno = d1.deptno
+            <![CDATA[SELECT e1.empno
+FROM emp e1, dept d1 where e1.deptno = d1.deptno
 and e1.deptno < 10 and d1.deptno < 15
 and exists (select * from emp e2 where e1.empno = e2.empno)]]>
         </Resource>
@@ -2670,4 +2681,545 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testWithInsideScalarSubqueryRex">
+        <Resource name="sql">
+            <![CDATA[select (
+ with dept2 as (select * from dept where deptno > 10) select count(*) from dept2) as c
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(C=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalProject(DEPTNO=[$0], NAME=[$1])
+      LogicalFilter(condition=[>($0, 10)])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testWithInsideWhereExistsRex">
+        <Resource name="sql">
+            <![CDATA[select * from emp
+where exists (
+  with dept2 as (select * from dept where dept.deptno >= emp.deptno)
+  select 1 from dept2 where deptno <= emp.deptno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[EXISTS({
+LogicalFilter(condition=[<=($0, $cor1.DEPTNO)])
+  LogicalProject(DEPTNO=[$0], NAME=[$1])
+    LogicalFilter(condition=[>=($0, $cor1.DEPTNO)])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})], variablesSet=[[$cor1]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testInUncorrelatedSubqueryInSelectRex">
+        <Resource name="sql">
+            <![CDATA[select name, deptno in (
+  select case when true then deptno else null end from emp)
+from dept]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(NAME=[$1], EXPR$1=[IN($0, {
+LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testNotInUncorrelatedSubqueryInSelectNotNullRex">
+        <Resource name="sql">
+            <![CDATA[select empno, deptno not in (
+  select deptno from dept)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], EXPR$1=[NOT(IN($7, {
+LogicalProject(DEPTNO=[$0])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+}))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testNotInUncorrelatedSubqueryRex">
+        <Resource name="sql">
+            <![CDATA[select empno from emp where deptno not in (select deptno from dept)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[NOT(IN($7, {
+LogicalProject(DEPTNO=[$0])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+}))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testNotInUncorrelatedSubqueryInSelectRex">
+        <Resource name="sql">
+            <![CDATA[select empno, deptno not in (
+  select case when true then deptno else null end from dept)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], EXPR$1=[NOT(IN($7, {
+LogicalProject(EXPR$0=[CASE(true, CAST($0):INTEGER, null)])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+}))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testInUncorrelatedSubqueryRex">
+        <Resource name="sql">
+            <![CDATA[select empno from emp where deptno in (select deptno from dept)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[IN($7, {
+LogicalProject(DEPTNO=[$0])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testWithInsideWhereExistsDecorrelateRex">
+        <Resource name="sql">
+            <![CDATA[select * from emp
+where exists (
+  with dept2 as (select * from dept where dept.deptno >= emp.deptno)
+  select 1 from dept2 where deptno <= emp.deptno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[EXISTS({
+LogicalFilter(condition=[<=($0, $cor1.DEPTNO)])
+  LogicalProject(DEPTNO=[$0], NAME=[$1])
+    LogicalFilter(condition=[>=($0, $cor1.DEPTNO)])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})], variablesSet=[[$cor1]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testCompositeInUncorrelatedSubqueryRex">
+        <Resource name="sql">
+            <![CDATA[select empno from emp where (empno, deptno) in (select deptno - 10, deptno from dept)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[IN($0, $7, {
+LogicalProject(EXPR$0=[-($0, 10)], DEPTNO=[$0])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJoinOnInSubQuery">
+        <Resource name="sql">
+            <![CDATA[select * from emp left join dept
+on emp.empno = 1
+or dept.deptno in (select deptno from emp where empno > 5)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[OR(=($0, 1), IN($9, {
+LogicalProject(DEPTNO=[$7])
+  LogicalFilter(condition=[>($0, 5)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}))], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJoinOnExists">
+        <Resource name="sql">
+            <![CDATA[select * from emp left join dept
+on emp.empno = 1
+or exists (select deptno from emp where empno > dept.deptno + 5)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[OR(=($0, 1), EXISTS({
+LogicalFilter(condition=[>($0, +($cor0.DEPTNO, 5))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}))], joinType=[left])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testInUncorrelatedSubqueryInHavingRex">
+        <Resource name="sql">
+            <![CDATA[select sum(sal) as s
+from emp
+group by deptno
+having count(*) > 2
+and deptno in (
+  select case when true then deptno else null end from emp)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(S=[$1])
+  LogicalFilter(condition=[AND(>($2, 2), IN($0, {
+LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+}))])
+    LogicalAggregate(group=[{0}], S=[SUM($1)], agg#1=[COUNT()])
+      LogicalProject(DEPTNO=[$7], SAL=[$5])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testUncorrelatedScalarSubqueryInGroupOrderRex">
+        <Resource name="sql">
+            <![CDATA[select sum(sal) as s
+from emp
+group by deptno
+order by (select case when true then deptno else null end from emp) desc,
+  count(*)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(S=[$0])
+  LogicalSort(sort0=[$1], sort1=[$2], dir0=[DESC], dir1=[ASC])
+    LogicalProject(S=[$1], EXPR$1=[$SCALAR_QUERY({
+LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})], EXPR$2=[$2])
+      LogicalAggregate(group=[{0}], S=[SUM($1)], agg#1=[COUNT()])
+        LogicalProject(DEPTNO=[$7], SAL=[$5])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testUncorrelatedScalarSubqueryInOrderRex">
+        <Resource name="sql">
+            <![CDATA[select ename
+from emp
+order by (select case when true then deptno else null end from emp) desc,
+  ename]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(ENAME=[$0])
+  LogicalSort(sort0=[$1], sort1=[$0], dir0=[DESC], dir1=[ASC])
+    LogicalProject(ENAME=[$1], EXPR$1=[$SCALAR_QUERY({
+LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testUncorrelatedScalarSubqueryInAggregateRex">
+        <Resource name="sql">
+            <![CDATA[select sum((select min(deptno) from emp)) as s
+from emp
+group by deptno
+]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(S=[$1])
+  LogicalAggregate(group=[{0}], S=[SUM($1)])
+    LogicalProject(DEPTNO=[$7], $f1=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[MIN($0)])
+  LogicalProject(DEPTNO=[$7])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testWhereInCorrelated">
+        <Resource name="sql">
+            <![CDATA[select empno from emp as e
+join dept as d using (deptno)
+where e.sal in (
+  select e2.sal from emp as e2 where e2.deptno > e.deptno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[IN($5, {
+LogicalProject(SAL=[$5])
+  LogicalFilter(condition=[>($7, $cor0.DEPTNO)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})], variablesSet=[[$cor0]])
+    LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testLateralDecorrelateRex">
+        <Resource name="sql">
+            <![CDATA[select * from emp,
+ LATERAL (select * from dept where emp.deptno=dept.deptno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
+  LogicalJoin(condition=[=($7, $11)], joinType=[inner])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalProject(DEPTNO=[$0], NAME=[$1], DEPTNO0=[$2])
+      LogicalJoin(condition=[=($2, $0)], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalAggregate(group=[{0}])
+          LogicalProject(DEPTNO=[$7])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExistsCorrelatedDecorrelateRex">
+        <Resource name="sql">
+            <![CDATA[select*from emp where exists (
+  select 1 from dept where emp.deptno=dept.deptno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[EXISTS({
+LogicalFilter(condition=[=($cor0.DEPTNO, $0)])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})], variablesSet=[[$cor0]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testNestedCorrelationsDecorrelatedRex">
+        <Resource name="sql">
+            <![CDATA[select *
+from (select 2+deptno d2, 3+deptno d3 from emp) e
+ where exists (select 1 from (select deptno+1 d1 from dept) d
+ where d1=e.d2 and exists (select 2 from (select deptno+4 d4, deptno+5 d5, deptno+6 d6 from dept)
+ where d4=d.d1 and d5=d.d1 and d6=e.d3))]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(D2=[$0], D3=[$1])
+  LogicalProject(D2=[$0], D3=[$1], $f0=[$4])
+    LogicalProject(D2=[$0], D3=[$1], D20=[CAST($2):INTEGER], D30=[$3], $f2=[CAST($4):BOOLEAN])
+      LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[inner])
+        LogicalProject(D2=[+(2, $7)], D3=[+(3, $7)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{0, 1}], agg#0=[MIN($2)])
+          LogicalProject(D2=[$1], D3=[$2], $f0=[$0])
+            LogicalProject($f0=[true], D2=[$1], D3=[$2])
+              LogicalProject(EXPR$0=[1], D2=[$3], D3=[$2])
+                LogicalJoin(condition=[=($0, $3)], joinType=[inner])
+                  LogicalFilter(condition=[IS NOT NULL($1)])
+                    LogicalProject(D1=[$0], $f0=[$4], D3=[$3])
+                      LogicalJoin(condition=[AND(=($0, $1), =($0, $2))], joinType=[left])
+                        LogicalProject(D1=[+($0, 1)])
+                          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+                        LogicalAggregate(group=[{0, 1, 2}], agg#0=[MIN($3)])
+                          LogicalProject(D1=[$1], D12=[$2], D3=[$3], $f0=[$0])
+                            LogicalProject($f0=[true], D1=[$1], D12=[$2], D3=[$3])
+                              LogicalProject(EXPR$0=[2], D1=[$3], D12=[$3], D3=[$4])
+                                LogicalJoin(condition=[AND(=($0, $3), =($1, $3), =($2, $4))], joinType=[inner])
+                                  LogicalProject(D4=[+($0, 4)], D5=[+($0, 5)], D6=[+($0, 6)])
+                                    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+                                  LogicalJoin(condition=[true], joinType=[inner])
+                                    LogicalAggregate(group=[{0}])
+                                      LogicalProject(D1=[+($0, 1)])
+                                        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+                                    LogicalAggregate(group=[{0}])
+                                      LogicalProject(D3=[$1])
+                                        LogicalProject(D2=[+(2, $7)], D3=[+(3, $7)])
+                                          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+                  LogicalAggregate(group=[{0}])
+                    LogicalProject(D2=[$0])
+                      LogicalProject(D2=[+(2, $7)], D3=[+(3, $7)])
+                        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJoinUnnestRex">
+        <Resource name="sql">
+            <![CDATA[select*from dept as d, unnest(multiset[d.deptno * 2])]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0], NAME=[$1], EXPR$0=[$2])
+  LogicalCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{0}])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    Uncollect
+      LogicalProject(EXPR$0=[$SLICE($0)])
+        Collect(field=[EXPR$0])
+          LogicalUnion(all=[true])
+            LogicalProject(EXPR$0=[*($cor0.DEPTNO, 2)])
+              LogicalValues(tuples=[[{ 0 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testCorrelationScalarAggAndFilterRex">
+        <Resource name="sql">
+            <![CDATA[SELECT e1.empno
+FROM emp e1, dept d1 where e1.deptno = d1.deptno
+and e1.deptno < 10 and d1.deptno < 15
+and e1.sal > (select avg(sal) from emp e2 where e1.empno = e2.empno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[AND(=($7, $9), <($7, 10), <($9, 15), >($5, $SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[AVG($0)])
+  LogicalProject(SAL=[$5])
+    LogicalFilter(condition=[=($cor0.EMPNO, $0)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+})))], variablesSet=[[$cor0]])
+    LogicalJoin(condition=[true], joinType=[inner])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testExistsCorrelatedLimitDecorrelateRex">
+        <Resource name="sql">
+            <![CDATA[select*from emp where exists (
+  select 1 from dept where emp.deptno=dept.deptno limit 1)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[EXISTS({
+LogicalSort(fetch=[1])
+  LogicalProject(EXPR$0=[1])
+    LogicalFilter(condition=[=($cor0.DEPTNO, $0)])
+      LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})], variablesSet=[[$cor0]])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testUnnestSelectRex">
+        <Resource name="sql">
+            <![CDATA[select*from unnest(select multiset[deptno] from dept)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[$0])
+  Uncollect
+    LogicalProject(EXPR$0=[$SLICE($2)])
+      LogicalCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{0}])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        Collect(field=[EXPR$0])
+          LogicalUnion(all=[true])
+            LogicalProject(EXPR$0=[$cor0.DEPTNO])
+              LogicalValues(tuples=[[{ 0 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testCorrelationExistsAndFilterRex">
+        <Resource name="sql">
+            <![CDATA[SELECT e1.empno
+FROM emp e1, dept d1 where e1.deptno = d1.deptno
+and e1.deptno < 10 and d1.deptno < 15
+and exists (select * from emp e2 where e1.empno = e2.empno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10], $f0=[$12])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10], EMPNO0=[CAST($11):INTEGER], $f1=[CAST($12):BOOLEAN])
+      LogicalJoin(condition=[=($0, $11)], joinType=[inner])
+        LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+          LogicalFilter(condition=[<($7, 10)])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+          LogicalFilter(condition=[<($0, 15)])
+            LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+        LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
+          LogicalProject(EMPNO0=[$1], $f0=[$0])
+            LogicalProject($f0=[true], EMPNO0=[$9])
+              LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], EMPNO0=[$9])
+                LogicalJoin(condition=[=($9, $0)], joinType=[inner])
+                  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+                  LogicalAggregate(group=[{0}])
+                    LogicalProject(EMPNO=[$0])
+                      LogicalJoin(condition=[=($7, $9)], joinType=[inner])
+                        LogicalFilter(condition=[<($7, 10)])
+                          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+                        LogicalFilter(condition=[<($0, 15)])
+                          LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testCorrelationJoinRex">
+        <Resource name="sql">
+            <![CDATA[select *,
+  multiset(select * from emp where deptno=dept.deptno) as empset
+from dept]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0], NAME=[$1], EMPSET=[$2])
+  LogicalCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{0}])
+    LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    Collect(field=[EXPR$0])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+        LogicalFilter(condition=[=($7, $cor0.DEPTNO)])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testMultisetOfColumnsRex">
+        <Resource name="sql">
+            <![CDATA[select 'abc',multiset[deptno,sal] from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=['abc'], EXPR$1=[$SLICE($9)])
+  LogicalCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{5, 7}])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    Collect(field=[EXPR$0])
+      LogicalUnion(all=[true])
+        LogicalProject(EXPR$0=[$cor0.DEPTNO])
+          LogicalValues(tuples=[[{ 0 }]])
+        LogicalProject(EXPR$0=[$cor0.SAL])
+          LogicalValues(tuples=[[{ 0 }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testNotExistsCorrelated">
+        <Resource name="sql">
+            <![CDATA[select * from emp where not exists (
+  select 1 from dept where emp.deptno=dept.deptno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[NOT(IS NOT NULL($9))])
+    LogicalCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{7}])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{}], agg#0=[MIN($0)])
+        LogicalProject($f0=[true])
+          LogicalProject(EXPR$0=[1])
+            LogicalFilter(condition=[=($cor0.DEPTNO, $0)])
+              LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/resources/sql/join.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/join.iq b/core/src/test/resources/sql/join.iq
index 357ddc5..aef20e0 100644
--- a/core/src/test/resources/sql/join.iq
+++ b/core/src/test/resources/sql/join.iq
@@ -195,15 +195,17 @@ join "scott".emp emp3 on (emp1.deptno + emp2.deptno = emp3.deptno + 10);
 
 !ok
 
-EnumerableCalc(expr#0..1=[{inputs}], DEPTNO0=[$t1], ENAME=[$t0])
-  EnumerableAggregate(group=[{1, 16}])
-    EnumerableJoin(condition=[=($8, $25)], joinType=[inner])
-      EnumerableCalc(expr#0..7=[{inputs}], expr#8=[10], expr#9=[+($t7, $t8)], proj#0..7=[{exprs}], $f8=[$t9])
+EnumerableCalc(expr#0..1=[{inputs}], DEPTNO=[$t1], ENAME=[$t0])
+  EnumerableAggregate(group=[{1, 3}])
+    EnumerableJoin(condition=[=($2, $4)], joinType=[inner])
+      EnumerableCalc(expr#0..7=[{inputs}], expr#8=[10], expr#9=[+($t7, $t8)], proj#0..1=[{exprs}], $f8=[$t9])
         EnumerableTableScan(table=[[scott, EMP]])
-      EnumerableCalc(expr#0..15=[{inputs}], expr#16=[+($t7, $t15)], expr#17=[CAST($t16):INTEGER], proj#0..15=[{exprs}], $f16=[$t17])
-        EnumerableJoin(condition=[=($7, $15)], joinType=[inner])
-          EnumerableTableScan(table=[[scott, EMP]])
-          EnumerableTableScan(table=[[scott, EMP]])
+      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[+($t1, $t3)], expr#5=[CAST($t4):INTEGER], DEPTNO=[$t1], $f16=[$t5])
+        EnumerableJoin(condition=[=($1, $3)], joinType=[inner])
+          EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], DEPTNO=[$t7])
+            EnumerableTableScan(table=[[scott, EMP]])
+          EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], DEPTNO=[$t7])
+            EnumerableTableScan(table=[[scott, EMP]])
 !plan
 
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 10d69fd..7612f22 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -257,6 +257,90 @@ EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t2):VARCHAR(1) CHARACTER SET
       EnumerableTableScan(table=[[hr, depts]])
 !plan
 
+# Un-correlated EXISTS
+select "deptno" from "hr"."depts"
+where exists (select 1 from "hr"."emps");
++--------+
+| deptno |
++--------+
+|     10 |
+|     30 |
+|     40 |
++--------+
+(3 rows)
+
+!ok
+EnumerableCalc(expr#0..1=[{inputs}], deptno=[$t1])
+  EnumerableJoin(condition=[true], joinType=[inner])
+    EnumerableCalc(expr#0=[{inputs}], expr#1=[IS NOT NULL($t0)], $f0=[$t0], $condition=[$t1])
+      EnumerableAggregate(group=[{}], agg#0=[MIN($0)])
+        EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f0=[$t5])
+          EnumerableTableScan(table=[[hr, emps]])
+    EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])
+      EnumerableTableScan(table=[[hr, depts]])
+!plan
+
+# Un-correlated NOT EXISTS
+select "deptno" from "hr"."depts"
+where not exists (select 1 from "hr"."emps");
++--------+
+| deptno |
++--------+
++--------+
+(0 rows)
+
+!ok
+EnumerableCalc(expr#0..1=[{inputs}], expr#2=[IS NOT NULL($t1)], expr#3=[NOT($t2)], deptno=[$t0], $condition=[$t3])
+  EnumerableJoin(condition=[true], joinType=[left])
+    EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])
+      EnumerableTableScan(table=[[hr, depts]])
+    EnumerableAggregate(group=[{}], agg#0=[MIN($0)])
+      EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f0=[$t5])
+        EnumerableTableScan(table=[[hr, emps]])
+!plan
+
+# Un-correlated EXISTS (table empty)
+select "deptno" from "hr"."depts"
+where exists (select 1 from "hr"."emps" where "empid" < 0);
++--------+
+| deptno |
++--------+
++--------+
+(0 rows)
+
+!ok
+EnumerableCalc(expr#0..1=[{inputs}], deptno=[$t1])
+  EnumerableJoin(condition=[true], joinType=[inner])
+    EnumerableCalc(expr#0=[{inputs}], expr#1=[IS NOT NULL($t0)], $f0=[$t0], $condition=[$t1])
+      EnumerableAggregate(group=[{}], agg#0=[MIN($0)])
+        EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], expr#6=[0], expr#7=[<($t0, $t6)], $f0=[$t5], $condition=[$t7])
+          EnumerableTableScan(table=[[hr, emps]])
+    EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])
+      EnumerableTableScan(table=[[hr, depts]])
+!plan
+
+# Un-correlated NOT EXISTS (table empty)
+select "deptno" from "hr"."depts"
+where not exists (select 1 from "hr"."emps" where "empid" < 0);
++--------+
+| deptno |
++--------+
+|     10 |
+|     30 |
+|     40 |
++--------+
+(3 rows)
+
+!ok
+EnumerableCalc(expr#0..1=[{inputs}], expr#2=[IS NOT NULL($t1)], expr#3=[NOT($t2)], deptno=[$t0], $condition=[$t3])
+  EnumerableJoin(condition=[true], joinType=[left])
+    EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])
+      EnumerableTableScan(table=[[hr, depts]])
+    EnumerableAggregate(group=[{}], agg#0=[MIN($0)])
+      EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], expr#6=[0], expr#7=[<($t0, $t6)], $f0=[$t5], $condition=[$t7])
+        EnumerableTableScan(table=[[hr, emps]])
+!plan
+
 # EXISTS
 select * from "hr"."emps"
 where exists (
@@ -273,10 +357,10 @@ where exists (
 !ok
 EnumerableSemiJoin(condition=[=($1, $5)], joinType=[inner])
   EnumerableTableScan(table=[[hr, emps]])
-  EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], deptno0=[$t0], $f0=[$t5])
-    EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
-      EnumerableAggregate(group=[{1}])
-        EnumerableTableScan(table=[[hr, emps]])
+  EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
+    EnumerableAggregate(group=[{1}])
+      EnumerableTableScan(table=[[hr, emps]])
+    EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])
       EnumerableTableScan(table=[[hr, depts]])
 !plan
 
@@ -297,11 +381,12 @@ EnumerableCalc(expr#0..6=[{inputs}], expr#7=[IS NOT NULL($t6)], expr#8=[NOT($t7)
   EnumerableJoin(condition=[=($1, $5)], joinType=[left])
     EnumerableTableScan(table=[[hr, emps]])
     EnumerableAggregate(group=[{1}], agg#0=[MIN($0)])
-      EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f0=[$t5], deptno0=[$t0])
+      EnumerableCalc(expr#0..1=[{inputs}], expr#2=[true], $f0=[$t2], deptno0=[$t0])
         EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
           EnumerableAggregate(group=[{1}])
             EnumerableTableScan(table=[[hr, emps]])
-          EnumerableTableScan(table=[[hr, depts]])
+          EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])
+            EnumerableTableScan(table=[[hr, depts]])
 !plan
 
 # NOT EXISTS .. OR NOT EXISTS
@@ -328,24 +413,26 @@ EnumerableCalc(expr#0..7=[{inputs}], expr#8=[IS NOT NULL($t5)], expr#9=[NOT($t8)
       EnumerableJoin(condition=[=($1, $5)], joinType=[left])
         EnumerableTableScan(table=[[hr, emps]])
         EnumerableAggregate(group=[{1}], agg#0=[MIN($0)])
-          EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f0=[$t5], deptno0=[$t0])
+          EnumerableCalc(expr#0..1=[{inputs}], expr#2=[true], $f0=[$t2], deptno0=[$t0])
             EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
               EnumerableAggregate(group=[{1}])
                 EnumerableTableScan(table=[[hr, emps]])
-              EnumerableTableScan(table=[[hr, depts]])
+              EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])
+                EnumerableTableScan(table=[[hr, depts]])
     EnumerableAggregate(group=[{1}], agg#0=[MIN($0)])
-      EnumerableCalc(expr#0..6=[{inputs}], expr#7=[true], $f0=[$t7], empid=[$t0])
-        EnumerableJoin(condition=[=($1, $6)], joinType=[inner])
+      EnumerableCalc(expr#0..3=[{inputs}], expr#4=[true], $f0=[$t4], empid=[$t0])
+        EnumerableJoin(condition=[=($1, $3)], joinType=[inner])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):INTEGER NOT NULL], proj#0..1=[{exprs}])
             EnumerableAggregate(group=[{0}])
-              EnumerableSemiJoin(condition=[=($1, $6)], joinType=[inner])
-                EnumerableTableScan(table=[[hr, emps]])
-                EnumerableCalc(expr#0..4=[{inputs}], expr#5=[true], $f0=[$t5], deptno0=[$t0])
-                  EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
-                    EnumerableAggregate(group=[{1}])
-                      EnumerableTableScan(table=[[hr, emps]])
+              EnumerableSemiJoin(condition=[=($1, $2)], joinType=[inner])
+                EnumerableCalc(expr#0..4=[{inputs}], proj#0..1=[{exprs}])
+                  EnumerableTableScan(table=[[hr, emps]])
+                EnumerableJoin(condition=[=($0, $1)], joinType=[inner])
+                  EnumerableAggregate(group=[{1}])
+                    EnumerableTableScan(table=[[hr, emps]])
+                  EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t0])
                     EnumerableTableScan(table=[[hr, depts]])
-          EnumerableCalc(expr#0..3=[{inputs}], expr#4=[90], expr#5=[+($t0, $t4)], proj#0..3=[{exprs}], $f4=[$t5])
+          EnumerableCalc(expr#0..3=[{inputs}], expr#4=[90], expr#5=[+($t0, $t4)], deptno=[$t0], $f1=[$t5])
             EnumerableTableScan(table=[[hr, depts]])
 !plan
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/core/src/test/resources/sql/subquery.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/subquery.iq b/core/src/test/resources/sql/subquery.iq
index b69b669..b9964b6 100644
--- a/core/src/test/resources/sql/subquery.iq
+++ b/core/src/test/resources/sql/subquery.iq
@@ -32,25 +32,24 @@ where t1.x not in (select t2.x from t2);
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..5=[{inputs}], expr#6=[0], expr#7=[=($t1, $t6)], expr#8=[false], expr#9=[IS NOT NULL($t5)], expr#10=[true], expr#11=[IS NULL($t3)], expr#12=[null], expr#13=[<($t2, $t1)], expr#14=[CASE($t7, $t8, $t9, $t10, $t11, $t12, $t13, $t12, $t8)], expr#15=[NOT($t14)], X=[$t0], $condition=[$t15])
-  EnumerableJoin(condition=[=($3, $4)], joinType=[left])
-    EnumerableCalc(expr#0..2=[{inputs}], $f0=[$t2], $f1=[$t0], $f2=[$t1], $f3=[$t2])
-      EnumerableJoin(condition=[true], joinType=[inner])
-        EnumerableAggregate(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
-          EnumerableUnion(all=[true])
-            EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
-              EnumerableValues(tuples=[[{ 0 }]])
-            EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
-              EnumerableValues(tuples=[[{ 0 }]])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[false], expr#8=[IS NOT NULL($t4)], expr#9=[true], expr#10=[IS NULL($t0)], expr#11=[null], expr#12=[<($t2, $t1)], expr#13=[CASE($t6, $t7, $t8, $t9, $t10, $t11, $t12, $t9, $t7)], expr#14=[NOT($t13)], EXPR$0=[$t0], $condition=[$t14])
+  EnumerableJoin(condition=[=($0, $3)], joinType=[left])
+    EnumerableJoin(condition=[true], joinType=[inner])
+      EnumerableUnion(all=[true])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
+          EnumerableValues(tuples=[[{ 0 }]])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[2], EXPR$0=[$t1])
+          EnumerableValues(tuples=[[{ 0 }]])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
+          EnumerableValues(tuples=[[{ 0 }]])
+      EnumerableAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
         EnumerableUnion(all=[true])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
             EnumerableValues(tuples=[[{ 0 }]])
-          EnumerableCalc(expr#0=[{inputs}], expr#1=[2], EXPR$0=[$t1])
-            EnumerableValues(tuples=[[{ 0 }]])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
             EnumerableValues(tuples=[[{ 0 }]])
-    EnumerableAggregate(group=[{0}], agg#0=[MIN($1)])
-      EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
+    EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
+      EnumerableAggregate(group=[{0}])
         EnumerableUnion(all=[true])
           EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
             EnumerableValues(tuples=[[{ 0 }]])
@@ -275,4 +274,47 @@ GROUP BY emp.deptno;
 
 !ok
 
+# Correlated IN sub-query in WHERE clause of JOIN
+select empno from "scott".emp as e
+join "scott".dept as d using (deptno)
+where e.job in (
+  select e2.job from "scott".emp as e2 where e2.deptno > e.deptno);
+ EMPNO
+-------
+  7369
+  7566
+  7782
+  7876
+  7934
+(5 rows)
+
+!ok
+EnumerableCalc(expr#0..5=[{inputs}], EMPNO=[$t0])
+  EnumerableJoin(condition=[=($2, $5)], joinType=[inner])
+    EnumerableCalc(expr#0..4=[{inputs}], EMPNO=[$t2], JOB=[$t3], DEPTNO=[$t4], JOB0=[$t0], DEPTNO0=[$t1])
+      EnumerableJoin(condition=[AND(=($1, $4), =($0, $3))], joinType=[inner])
+        EnumerableCalc(expr#0..1=[{inputs}], JOB=[$t1], DEPTNO=[$t0])
+          EnumerableAggregate(group=[{0, 2}])
+            EnumerableCalc(expr#0..3=[{inputs}], expr#4=[>($t3, $t0)], proj#0..3=[{exprs}], $condition=[$t4])
+              EnumerableJoin(condition=[true], joinType=[inner])
+                EnumerableAggregate(group=[{7}])
+                  EnumerableTableScan(table=[[scott, EMP]])
+                EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], JOB=[$t2], DEPTNO=[$t7])
+                  EnumerableTableScan(table=[[scott, EMP]])
+        EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], JOB=[$t2], DEPTNO=[$t7])
+          EnumerableTableScan(table=[[scott, EMP]])
+    EnumerableCalc(expr#0..2=[{inputs}], DEPTNO=[$t0])
+      EnumerableTableScan(table=[[scott, DEPT]])
+!plan
+
+!if (fixed.calcite1045) {
+# Correlated NOT IN sub-query in WHERE clause of JOIN
+select empno from "scott".emp as e
+join "scott".dept as d using (deptno)
+where e.job not in (
+  select e2.job from "scott".emp as e2 where e2.deptno > e.deptno);
+!ok
+!plan
+!}
+
 # End subquery.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index dbf2974..7bc9bc3 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -173,16 +173,17 @@ functions, or a combination of constants and aggregate
 functions. Aggregate and grouping functions may only appear in an
 aggregate query, and only in a SELECT, HAVING or ORDER BY clause.
 
-A scalar sub-query is a sub-query used as an expression. It can occur
-in most places where an expression can occur (such as the SELECT
-clause, WHERE clause, or as an argument to an aggregate
-function). If the sub-query returns no rows, the value is NULL; if it
+A scalar sub-query is a sub-query used as an expression.
+If the sub-query returns no rows, the value is NULL; if it
 returns more than one row, it is an error.
 
-A sub-query can occur in the FROM clause of a query and also in IN
-and EXISTS expressions.  A sub-query that occurs in IN and
-EXISTS expressions may be correlated; that is, refer to tables in
-the FROM clause of an enclosing query.
+IN, EXISTS and scalar sub-queries can occur
+in any place where an expression can occur (such as the SELECT clause,
+WHERE clause, ON clause of a JOIN, or as an argument to an aggregate
+function).
+
+An IN, EXISTS or scalar sub-query may be correlated; that is, it
+may refer to tables in the FROM clause of an enclosing query.
 
 ## Keywords
 


[08/50] [abbrv] calcite git commit: Add benchmark of Parser.create(sql).parseQuery()

Posted by jh...@apache.org.
Add benchmark of Parser.create(sql).parseQuery()

Closes #176


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

Branch: refs/heads/branch-release
Commit: 4a29b3ccf64e9a7d1534f540fd7f6d3d813605a0
Parents: b94a00e
Author: Vladimir Sitnikov <si...@gmail.com>
Authored: Sat Jan 9 13:36:33 2016 +0300
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:46:49 2016 -0800

----------------------------------------------------------------------
 .../apache/calcite/sql/parser/SqlParser.java    |  25 +-
 pom.xml                                         |   4 +-
 ubenchmark/pom.xml                              |   5 +-
 .../java/org/apache/calcite/StatementTest.java  | 264 -------------------
 .../benchmarks/FlightRecorderProfiler.java      |  87 ++++++
 .../calcite/benchmarks/ParserBenchmark.java     | 121 +++++++++
 .../calcite/benchmarks/PreconditionTest.java    |  56 ++++
 .../calcite/benchmarks/StatementTest.java       | 264 +++++++++++++++++++
 .../apache/calcite/benchmarks/package-info.java |  26 ++
 9 files changed, 575 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
index 4c14395..e1bda73 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
@@ -133,22 +133,27 @@ public class SqlParser {
   }
 
   /**
+   * Parses a <code>SELECT</code> statement and reuses parser.
+   *
+   * @param sql sql to parse
+   * @return A {@link org.apache.calcite.sql.SqlSelect} for a regular <code>
+   * SELECT</code> statement; a {@link org.apache.calcite.sql.SqlBinaryOperator}
+   * for a <code>UNION</code>, <code>INTERSECT</code>, or <code>EXCEPT</code>.
+   * @throws SqlParseException if there is a parse error
+   */
+  public SqlNode parseQuery(String sql) throws SqlParseException {
+    parser.ReInit(new StringReader(sql));
+    return parseQuery();
+  }
+
+  /**
    * Parses an SQL statement.
    *
    * @return top-level SqlNode representing stmt
    * @throws SqlParseException if there is a parse error
    */
   public SqlNode parseStmt() throws SqlParseException {
-    try {
-      return parser.parseSqlStmtEof();
-    } catch (Throwable ex) {
-      if ((ex instanceof CalciteContextException)
-          && (originalInput != null)) {
-        ((CalciteContextException) ex).setOriginalStatement(
-            originalInput);
-      }
-      throw parser.normalizeException(ex);
-    }
+    return parseQuery();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9ac5596..3253f25 100644
--- a/pom.xml
+++ b/pom.xml
@@ -295,12 +295,12 @@ limitations under the License.
       <dependency>
         <groupId>org.openjdk.jmh</groupId>
         <artifactId>jmh-core</artifactId>
-        <version>0.7.1</version>
+        <version>1.11.2</version>
       </dependency>
       <dependency>
         <groupId>org.openjdk.jmh</groupId>
         <artifactId>jmh-generator-annprocess</artifactId>
-        <version>0.7.1</version>
+        <version>1.11.2</version>
       </dependency>
       <dependency>
         <groupId>sqlline</groupId>

http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/ubenchmark/pom.xml
----------------------------------------------------------------------
diff --git a/ubenchmark/pom.xml b/ubenchmark/pom.xml
index 1f1752c..e50d4dd 100644
--- a/ubenchmark/pom.xml
+++ b/ubenchmark/pom.xml
@@ -55,6 +55,10 @@ limitations under the License.
       <artifactId>jmh-generator-annprocess</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
   </dependencies>
 
   <build>
@@ -64,7 +68,6 @@ limitations under the License.
         <configuration>
           <source>1.6</source>
           <target>1.6</target>
-          <compilerArgument>-proc:none</compilerArgument>
         </configuration>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/ubenchmark/src/main/java/org/apache/calcite/StatementTest.java
----------------------------------------------------------------------
diff --git a/ubenchmark/src/main/java/org/apache/calcite/StatementTest.java b/ubenchmark/src/main/java/org/apache/calcite/StatementTest.java
deleted file mode 100644
index b8a7d2f..0000000
--- a/ubenchmark/src/main/java/org/apache/calcite/StatementTest.java
+++ /dev/null
@@ -1,264 +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;
-
-import org.apache.calcite.adapter.java.ReflectiveSchema;
-import org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.calcite.schema.SchemaPlus;
-
-import org.openjdk.jmh.annotations.BenchmarkMode;
-import org.openjdk.jmh.annotations.GenerateMicroBenchmark;
-import org.openjdk.jmh.annotations.Level;
-import org.openjdk.jmh.annotations.Mode;
-import org.openjdk.jmh.annotations.Scope;
-import org.openjdk.jmh.annotations.Setup;
-import org.openjdk.jmh.annotations.State;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-/**
- * Compares {@link java.sql.Statement} vs {@link java.sql.PreparedStatement}.
- *
- * <p>This package contains micro-benchmarks to test calcite performance.
- *
- * <p>To run this and other benchmarks:
- *
- * <blockquote>
- *   <code>mvn package &amp;&amp;
- *   java -jar ./target/ubenchmarks.jar -wi 5 -i 5 -f 1</code>
- * </blockquote>
- *
- * <p>To run with profiling:
- *
- * <blockquote>
- *   <code>java -Djmh.stack.lines=10 -jar ./target/ubenchmarks.jar
- *     -prof hs_comp,hs_gc,stack -f 1 -wi 5</code>
- * </blockquote>
- */
-public class StatementTest {
-
-  /**
-   * Connection to be used during tests.
-   */
-  @State(Scope.Thread)
-  @BenchmarkMode(Mode.AverageTime)
-  public static class HrConnection {
-    Connection con;
-    int id;
-    HrSchema hr = new HrSchema();
-    Random rnd = new Random();
-    {
-      try {
-        Class.forName("org.apache.calcite.jdbc.Driver");
-      } catch (ClassNotFoundException e) {
-        throw new IllegalStateException(e);
-      }
-      Connection connection;
-
-      try {
-        Properties info = new Properties();
-        info.put("lex", "JAVA");
-        info.put("quoting", "DOUBLE_QUOTE");
-        connection = DriverManager.getConnection("jdbc:calcite:", info);
-      } catch (SQLException e) {
-        throw new IllegalStateException(e);
-      }
-      CalciteConnection calciteConnection;
-      try {
-        calciteConnection = connection.unwrap(CalciteConnection.class);
-      } catch (SQLException e) {
-        throw new IllegalStateException(e);
-      }
-      final SchemaPlus rootSchema = calciteConnection.getRootSchema();
-      rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
-      try {
-        calciteConnection.setSchema("hr");
-      } catch (SQLException e) {
-        throw new IllegalStateException(e);
-      }
-      con = connection;
-    }
-
-    @Setup(Level.Iteration)
-    public void pickEmployee() {
-      id = hr.emps[rnd.nextInt(4)].empid;
-    }
-  }
-
-  /**
-   * Tests performance of reused execution of prepared statement.
-   */
-  public static class HrPreparedStatement extends HrConnection {
-    PreparedStatement ps;
-    {
-      try {
-        ps = con.prepareStatement("select name from emps where empid = ?");
-      } catch (SQLException e) {
-        throw new IllegalStateException(e);
-      }
-    }
-  }
-
-  @GenerateMicroBenchmark
-  public String prepareBindExecute(HrConnection state) throws SQLException {
-    Connection con = state.con;
-    Statement st = null;
-    ResultSet rs = null;
-    String ename = null;
-    try {
-      final PreparedStatement ps =
-          con.prepareStatement("select name from emps where empid = ?");
-      st = ps;
-      ps.setInt(1, state.id);
-      rs = ps.executeQuery();
-      rs.next();
-      ename = rs.getString(1);
-    } finally {
-      close(rs, st);
-    }
-    return ename;
-  }
-
-  @GenerateMicroBenchmark
-  public String bindExecute(HrPreparedStatement state)
-      throws SQLException {
-    PreparedStatement st = state.ps;
-    ResultSet rs = null;
-    String ename = null;
-    try {
-      st.setInt(1, state.id);
-      rs = st.executeQuery();
-      rs.next();
-      ename = rs.getString(1);
-    } finally {
-      close(rs, null); // Statement is not closed
-    }
-    return ename;
-  }
-
-  @GenerateMicroBenchmark
-  public String executeQuery(HrConnection state) throws SQLException {
-    Connection con = state.con;
-    Statement st = null;
-    ResultSet rs = null;
-    String ename = null;
-    try {
-      st = con.createStatement();
-      rs = st.executeQuery("select name from emps where empid = " + state.id);
-      rs.next();
-      ename = rs.getString(1);
-    } finally {
-      close(rs, st);
-    }
-    return ename;
-  }
-
-  @GenerateMicroBenchmark
-  public String forEach(HrConnection state) throws SQLException {
-    final Employee[] emps = state.hr.emps;
-    for (Employee emp : emps) {
-      if (emp.empid == state.id) {
-        return emp.name;
-      }
-    }
-    return null;
-  }
-
-  private static void close(ResultSet rs, Statement st) {
-    if (rs != null) {
-      try { rs.close(); } catch (SQLException e) { /**/ }
-    }
-    if (st != null) {
-      try { st.close(); } catch (SQLException e) { /**/ }
-    }
-  }
-
-  /** Pojo schema containing "emps" and "depts" tables. */
-  public static class HrSchema {
-    @Override public String toString() {
-      return "HrSchema";
-    }
-
-    public final Employee[] emps = {
-      new Employee(100, 10, "Bill", 10000, 1000),
-      new Employee(200, 20, "Eric", 8000, 500),
-      new Employee(150, 10, "Sebastian", 7000, null),
-      new Employee(110, 10, "Theodore", 11500, 250),
-    };
-    public final Department[] depts = {
-      new Department(10, "Sales", Arrays.asList(emps[0], emps[2])),
-      new Department(30, "Marketing", Collections.<Employee>emptyList()),
-      new Department(40, "HR", Collections.singletonList(emps[1])),
-    };
-  }
-
-  /** Employee record. */
-  public static class Employee {
-    public final int empid;
-    public final int deptno;
-    public final String name;
-    public final float salary;
-    public final Integer commission;
-
-    public Employee(int empid, int deptno, String name, float salary,
-        Integer commission) {
-      this.empid = empid;
-      this.deptno = deptno;
-      this.name = name;
-      this.salary = salary;
-      this.commission = commission;
-    }
-
-    public String toString() {
-      return "Employee [empid: " + empid + ", deptno: " + deptno
-          + ", name: " + name + "]";
-    }
-  }
-
-  /** Department record. */
-  public static class Department {
-    public final int deptno;
-    public final String name;
-    public final List<Employee> employees;
-
-    public Department(
-        int deptno, String name, List<Employee> employees) {
-      this.deptno = deptno;
-      this.name = name;
-      this.employees = employees;
-    }
-
-
-    public String toString() {
-      return "Department [deptno: " + deptno + ", name: " + name
-          + ", employees: " + employees + "]";
-    }
-  }
-
-}
-
-// End StatementTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/ubenchmark/src/main/java/org/apache/calcite/benchmarks/FlightRecorderProfiler.java
----------------------------------------------------------------------
diff --git a/ubenchmark/src/main/java/org/apache/calcite/benchmarks/FlightRecorderProfiler.java b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/FlightRecorderProfiler.java
new file mode 100644
index 0000000..451c004
--- /dev/null
+++ b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/FlightRecorderProfiler.java
@@ -0,0 +1,87 @@
+/*
+ * 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.benchmarks;
+
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.infra.IterationParams;
+import org.openjdk.jmh.profile.ExternalProfiler;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.Result;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Captures Flight Recorder log.
+ * Note: Flight Recorder is available in OracleJDK only.
+ * Usage of Flight Recorder in production requires a LICENSE FEE, however Flight Recorder is free
+ * for use in test systems.
+ * It is assumed you would not use Calcite benchmarks for running a production system, thus it is
+ * believed to be safe.
+ */
+public class FlightRecorderProfiler implements ExternalProfiler {
+  @Override public Collection<String> addJVMInvokeOptions(BenchmarkParams params) {
+    return Collections.emptyList();
+  }
+
+  @Override public Collection<String> addJVMOptions(BenchmarkParams params) {
+    StringBuilder sb = new StringBuilder();
+    for (String param : params.getParamsKeys()) {
+      if (sb.length() != 0) {
+        sb.append('-');
+      }
+      sb.append(param).append('-').append(params.getParam(param));
+    }
+
+    long duration =
+        getDurationSeconds(params.getWarmup()) + getDurationSeconds(params.getMeasurement());
+    return Arrays.asList(
+        "-XX:+UnlockCommercialFeatures", "-XX:+FlightRecorder",
+        "-XX:StartFlightRecording=settings=profile,duration=" + duration + "s,filename="
+            + params.getBenchmark() + "_" + sb + ".jfr");
+  }
+
+  private long getDurationSeconds(IterationParams warmup) {
+    return warmup.getTime().convertTo(TimeUnit.SECONDS) * warmup.getCount();
+  }
+
+  @Override public void beforeTrial(BenchmarkParams benchmarkParams) {
+
+  }
+
+  @Override public Collection<? extends Result> afterTrial(BenchmarkResult br, long pid,
+      File stdOut, File stdErr) {
+    return Collections.emptyList();
+  }
+
+  @Override public boolean allowPrintOut() {
+    return true;
+  }
+
+  @Override public boolean allowPrintErr() {
+    return true;
+  }
+
+  @Override public String getDescription() {
+    return "Collects Java Flight Recorder profile";
+  }
+}
+
+// End FlightRecorderProfiler.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/ubenchmark/src/main/java/org/apache/calcite/benchmarks/ParserBenchmark.java
----------------------------------------------------------------------
diff --git a/ubenchmark/src/main/java/org/apache/calcite/benchmarks/ParserBenchmark.java b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/ParserBenchmark.java
new file mode 100644
index 0000000..90e4e8a
--- /dev/null
+++ b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/ParserBenchmark.java
@@ -0,0 +1,121 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.profile.GCProfiler;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Benchmarks JavaCC-generated SQL parser
+ */
+@Fork(value = 1, jvmArgsPrepend = "-Xmx128m")
+@Measurement(iterations = 7, time = 1, timeUnit = TimeUnit.SECONDS)
+@Warmup(iterations = 7, time = 1, timeUnit = TimeUnit.SECONDS)
+@State(Scope.Thread)
+@Threads(1)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MICROSECONDS)
+public class ParserBenchmark {
+
+  @Param({ "1000" })
+  int length;
+
+  @Param({ "true" })
+  boolean comments;
+
+  String sql;
+  SqlParser parser;
+
+  @Setup
+  public void setup() throws SqlParseException {
+    StringBuilder sb = new StringBuilder((int) (length * 1.2));
+    sb.append("select 1");
+    Random rnd = new Random();
+    rnd.setSeed(424242);
+    for (; sb.length() < length;) {
+      for (int i = 0; i < 7 && sb.length() < length; i++) {
+        sb.append(", ");
+        switch (rnd.nextInt(3)) {
+        case 0:
+          sb.append("?");
+          break;
+        case 1:
+          sb.append(rnd.nextInt());
+          break;
+        case 2:
+          sb.append('\'').append(rnd.nextLong()).append(rnd.nextLong())
+              .append('\'');
+          break;
+        }
+      }
+      if (comments && sb.length() < length) {
+        sb.append("// sb.append('\\'').append(rnd.nextLong()).append(rnd.nextLong()).append(rnd"
+            + ".nextLong())");
+      }
+      sb.append('\n');
+    }
+    sb.append(" from dual");
+    parser = SqlParser.create("values(1)");
+    sql = sb.toString();
+  }
+
+  @Benchmark
+  public SqlNode parseCached() throws SqlParseException {
+    return parser.parseQuery(sql);
+  }
+
+  @Benchmark
+  public SqlNode parseNonCached() throws SqlParseException {
+    return SqlParser.create(sql).parseQuery();
+  }
+
+  public static void main(String[] args) throws RunnerException {
+    Options opt = new OptionsBuilder()
+        .include(ParserBenchmark.class.getSimpleName())
+        .addProfiler(GCProfiler.class)
+        .addProfiler(FlightRecorderProfiler.class)
+        .detectJvmArgs()
+        .build();
+
+    new Runner(opt).run();
+  }
+
+}
+
+// End ParserBenchmark.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/ubenchmark/src/main/java/org/apache/calcite/benchmarks/PreconditionTest.java
----------------------------------------------------------------------
diff --git a/ubenchmark/src/main/java/org/apache/calcite/benchmarks/PreconditionTest.java b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/PreconditionTest.java
new file mode 100644
index 0000000..c150a49
--- /dev/null
+++ b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/PreconditionTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.benchmarks;
+
+import com.google.common.base.Preconditions;
+
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.profile.GCProfiler;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+/**
+ * Checks if silent precondition has noticeable overhead
+ */
+@BenchmarkMode(Mode.AverageTime)
+@State(Scope.Benchmark)
+public class PreconditionTest {
+  boolean fire = false;
+  String param = "world";
+
+  public void testPrecondition() {
+    Preconditions.checkState(fire, "Hello %s", param);
+  }
+
+  public static void main(String[] args) throws RunnerException {
+    Options opt = new OptionsBuilder()
+        .include(PreconditionTest.class.getSimpleName())
+        .addProfiler(GCProfiler.class)
+        .detectJvmArgs()
+        .build();
+
+    new Runner(opt).run();
+  }
+
+}
+
+// End PreconditionTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/ubenchmark/src/main/java/org/apache/calcite/benchmarks/StatementTest.java
----------------------------------------------------------------------
diff --git a/ubenchmark/src/main/java/org/apache/calcite/benchmarks/StatementTest.java b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/StatementTest.java
new file mode 100644
index 0000000..d716fdf
--- /dev/null
+++ b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/StatementTest.java
@@ -0,0 +1,264 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.SchemaPlus;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+/**
+ * Compares {@link java.sql.Statement} vs {@link java.sql.PreparedStatement}.
+ *
+ * <p>This package contains micro-benchmarks to test calcite performance.
+ *
+ * <p>To run this and other benchmarks:
+ *
+ * <blockquote>
+ *   <code>mvn package &amp;&amp;
+ *   java -jar ./target/ubenchmarks.jar -wi 5 -i 5 -f 1</code>
+ * </blockquote>
+ *
+ * <p>To run with profiling:
+ *
+ * <blockquote>
+ *   <code>java -Djmh.stack.lines=10 -jar ./target/ubenchmarks.jar
+ *     -prof hs_comp,hs_gc,stack -f 1 -wi 5</code>
+ * </blockquote>
+ */
+public class StatementTest {
+
+  /**
+   * Connection to be used during tests.
+   */
+  @State(Scope.Thread)
+  @BenchmarkMode(Mode.AverageTime)
+  public static class HrConnection {
+    Connection con;
+    int id;
+    HrSchema hr = new HrSchema();
+    Random rnd = new Random();
+    {
+      try {
+        Class.forName("org.apache.calcite.jdbc.Driver");
+      } catch (ClassNotFoundException e) {
+        throw new IllegalStateException(e);
+      }
+      Connection connection;
+
+      try {
+        Properties info = new Properties();
+        info.put("lex", "JAVA");
+        info.put("quoting", "DOUBLE_QUOTE");
+        connection = DriverManager.getConnection("jdbc:calcite:", info);
+      } catch (SQLException e) {
+        throw new IllegalStateException(e);
+      }
+      CalciteConnection calciteConnection;
+      try {
+        calciteConnection = connection.unwrap(CalciteConnection.class);
+      } catch (SQLException e) {
+        throw new IllegalStateException(e);
+      }
+      final SchemaPlus rootSchema = calciteConnection.getRootSchema();
+      rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
+      try {
+        calciteConnection.setSchema("hr");
+      } catch (SQLException e) {
+        throw new IllegalStateException(e);
+      }
+      con = connection;
+    }
+
+    @Setup(Level.Iteration)
+    public void pickEmployee() {
+      id = hr.emps[rnd.nextInt(4)].empid;
+    }
+  }
+
+  /**
+   * Tests performance of reused execution of prepared statement.
+   */
+  public static class HrPreparedStatement extends HrConnection {
+    PreparedStatement ps;
+    {
+      try {
+        ps = con.prepareStatement("select name from emps where empid = ?");
+      } catch (SQLException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+  }
+
+  @Benchmark
+  public String prepareBindExecute(HrConnection state) throws SQLException {
+    Connection con = state.con;
+    Statement st = null;
+    ResultSet rs = null;
+    String ename = null;
+    try {
+      final PreparedStatement ps =
+          con.prepareStatement("select name from emps where empid = ?");
+      st = ps;
+      ps.setInt(1, state.id);
+      rs = ps.executeQuery();
+      rs.next();
+      ename = rs.getString(1);
+    } finally {
+      close(rs, st);
+    }
+    return ename;
+  }
+
+  @Benchmark
+  public String bindExecute(HrPreparedStatement state)
+      throws SQLException {
+    PreparedStatement st = state.ps;
+    ResultSet rs = null;
+    String ename = null;
+    try {
+      st.setInt(1, state.id);
+      rs = st.executeQuery();
+      rs.next();
+      ename = rs.getString(1);
+    } finally {
+      close(rs, null); // Statement is not closed
+    }
+    return ename;
+  }
+
+  @Benchmark
+  public String executeQuery(HrConnection state) throws SQLException {
+    Connection con = state.con;
+    Statement st = null;
+    ResultSet rs = null;
+    String ename = null;
+    try {
+      st = con.createStatement();
+      rs = st.executeQuery("select name from emps where empid = " + state.id);
+      rs.next();
+      ename = rs.getString(1);
+    } finally {
+      close(rs, st);
+    }
+    return ename;
+  }
+
+  @Benchmark
+  public String forEach(HrConnection state) throws SQLException {
+    final Employee[] emps = state.hr.emps;
+    for (Employee emp : emps) {
+      if (emp.empid == state.id) {
+        return emp.name;
+      }
+    }
+    return null;
+  }
+
+  private static void close(ResultSet rs, Statement st) {
+    if (rs != null) {
+      try { rs.close(); } catch (SQLException e) { /**/ }
+    }
+    if (st != null) {
+      try { st.close(); } catch (SQLException e) { /**/ }
+    }
+  }
+
+  /** Pojo schema containing "emps" and "depts" tables. */
+  public static class HrSchema {
+    @Override public String toString() {
+      return "HrSchema";
+    }
+
+    public final Employee[] emps = {
+      new Employee(100, 10, "Bill", 10000, 1000),
+      new Employee(200, 20, "Eric", 8000, 500),
+      new Employee(150, 10, "Sebastian", 7000, null),
+      new Employee(110, 10, "Theodore", 11500, 250),
+    };
+    public final Department[] depts = {
+      new Department(10, "Sales", Arrays.asList(emps[0], emps[2])),
+      new Department(30, "Marketing", Collections.<Employee>emptyList()),
+      new Department(40, "HR", Collections.singletonList(emps[1])),
+    };
+  }
+
+  /** Employee record. */
+  public static class Employee {
+    public final int empid;
+    public final int deptno;
+    public final String name;
+    public final float salary;
+    public final Integer commission;
+
+    public Employee(int empid, int deptno, String name, float salary,
+        Integer commission) {
+      this.empid = empid;
+      this.deptno = deptno;
+      this.name = name;
+      this.salary = salary;
+      this.commission = commission;
+    }
+
+    public String toString() {
+      return "Employee [empid: " + empid + ", deptno: " + deptno
+          + ", name: " + name + "]";
+    }
+  }
+
+  /** Department record. */
+  public static class Department {
+    public final int deptno;
+    public final String name;
+    public final List<Employee> employees;
+
+    public Department(
+        int deptno, String name, List<Employee> employees) {
+      this.deptno = deptno;
+      this.name = name;
+      this.employees = employees;
+    }
+
+
+    public String toString() {
+      return "Department [deptno: " + deptno + ", name: " + name
+          + ", employees: " + employees + "]";
+    }
+  }
+
+}
+
+// End StatementTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4a29b3cc/ubenchmark/src/main/java/org/apache/calcite/benchmarks/package-info.java
----------------------------------------------------------------------
diff --git a/ubenchmark/src/main/java/org/apache/calcite/benchmarks/package-info.java b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/package-info.java
new file mode 100644
index 0000000..3146cae
--- /dev/null
+++ b/ubenchmark/src/main/java/org/apache/calcite/benchmarks/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.
+ */
+
+/**
+ * JMH benchmarks for Calcite
+ */
+@PackageMarker
+package org.apache.calcite.benchmarks;
+
+import org.apache.calcite.avatica.util.PackageMarker;
+
+// End package-info.java


[24/50] [abbrv] calcite git commit: [CALCITE-816] Represent sub-query as a RexNode

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/505a9064/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 2f1d6b9..2812851 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -31,12 +31,16 @@ import org.apache.calcite.rel.BiRel;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttleImpl;
-import org.apache.calcite.rel.RelVisitor;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Correlate;
 import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalCorrelate;
 import org.apache.calcite.rel.logical.LogicalFilter;
@@ -58,6 +62,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlExplainLevel;
@@ -67,20 +72,25 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlCountAggFunction;
 import org.apache.calcite.sql.fun.SqlSingleValueAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.ReflectUtil;
-import org.apache.calcite.util.ReflectiveVisitDispatcher;
 import org.apache.calcite.util.ReflectiveVisitor;
+import org.apache.calcite.util.Stacks;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
 import org.apache.calcite.util.trace.CalciteTrace;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
@@ -96,6 +106,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -127,10 +138,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
   //~ Instance fields --------------------------------------------------------
 
+  private final RelBuilder relBuilder;
+
   // map built during translation
   private CorelMap cm;
 
-  private final DecorrelateRelVisitor decorrelateVisitor;
+  private final ReflectUtil.MethodDispatcher<Frame> dispatcher =
+      ReflectUtil.createMethodDispatcher(Frame.class, this, "decorrelateRel",
+          RelNode.class);
 
   private final RexBuilder rexBuilder;
 
@@ -139,31 +154,24 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
   private final Context context;
 
-  // maps built during decorrelation
-  private final Map<RelNode, RelNode> mapOldToNewRel = Maps.newHashMap();
-
-  // map rel to all the newly created correlated variables in its output
-  private final Map<RelNode, SortedMap<Correlation, Integer>>
-  mapNewRelToMapCorVarToOutputPos = Maps.newHashMap();
-
-  // another map to map old input positions to new input positions
-  // this is from the view point of the parent rel of a new rel.
-  private final Map<RelNode, Map<Integer, Integer>>
-  mapNewRelToMapOldToNewOutputPos = Maps.newHashMap();
+  /** Built during decorrelation, of rel to all the newly created correlated
+   * variables in its output, and to map old input positions to new input
+   * positions. This is from the view point of the parent rel of a new rel. */
+  private final Map<RelNode, Frame> map = new HashMap<>();
 
   private final HashSet<LogicalCorrelate> generatedCorRels = Sets.newHashSet();
 
   //~ Constructors -----------------------------------------------------------
 
   private RelDecorrelator(
-      RexBuilder rexBuilder,
+      RelOptCluster cluster,
       CorelMap cm,
       Context context) {
     this.cm = cm;
-    this.rexBuilder = rexBuilder;
+    this.rexBuilder = cluster.getRexBuilder();
     this.context = context;
+    relBuilder = RelFactories.LOGICAL_BUILDER.create(cluster, null);
 
-    decorrelateVisitor = new DecorrelateRelVisitor();
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -178,18 +186,16 @@ public class RelDecorrelator implements ReflectiveVisitor {
    * {@link org.apache.calcite.rel.logical.LogicalCorrelate} instances removed
    */
   public static RelNode decorrelateQuery(RelNode rootRel) {
-    final CorelMap corelMap = CorelMap.build(rootRel);
+    final CorelMap corelMap = new CorelMapBuilder().build(rootRel);
     if (!corelMap.hasCorrelation()) {
       return rootRel;
     }
 
     final RelOptCluster cluster = rootRel.getCluster();
-    final RexBuilder rexBuilder = cluster.getRexBuilder();
     final RelDecorrelator decorrelator =
-        new RelDecorrelator(rexBuilder, corelMap,
+        new RelDecorrelator(cluster, corelMap,
             cluster.getPlanner().getContext());
 
-
     RelNode newRootRel = decorrelator.removeCorrelationViaRule(rootRel);
 
     if (SQL2REL_LOGGER.isLoggable(Level.FINE)) {
@@ -211,7 +217,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
   private void setCurrent(RelNode root, LogicalCorrelate corRel) {
     currentRel = corRel;
     if (corRel != null) {
-      cm = CorelMap.build(Util.first(root, corRel));
+      cm = new CorelMapBuilder().build(Util.first(root, corRel));
     }
   }
 
@@ -231,13 +237,10 @@ public class RelDecorrelator implements ReflectiveVisitor {
     root = planner.findBestExp();
 
     // Perform decorrelation.
-    mapOldToNewRel.clear();
-    mapNewRelToMapCorVarToOutputPos.clear();
-    mapNewRelToMapOldToNewOutputPos.clear();
-
-    decorrelateVisitor.visit(root, 0, null);
+    map.clear();
 
-    if (mapOldToNewRel.containsKey(root)) {
+    final Frame frame = getInvoke(root, null);
+    if (frame != null) {
       // has been rewritten; apply rules post-decorrelation
       final HepProgram program2 = HepProgram.builder()
           .addRuleInstance(FilterJoinRule.FILTER_ON_JOIN)
@@ -245,7 +248,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           .build();
 
       final HepPlanner planner2 = createPlanner(program2);
-      final RelNode newRoot = mapOldToNewRel.get(root);
+      final RelNode newRoot = frame.r;
       planner2.setRoot(newRoot);
       return planner2.findBestExp();
     }
@@ -265,7 +268,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           LogicalCorrelate oldCor = (LogicalCorrelate) oldNode;
           CorrelationId c = oldCor.getCorrelationId();
           if (cm.mapCorVarToCorRel.get(c) == oldNode) {
-            cm.mapCorVarToCorRel.put(c, (LogicalCorrelate) newNode);
+            cm.mapCorVarToCorRel.put(c, newNode);
           }
 
           if (generatedCorRels.contains(oldNode)) {
@@ -298,9 +301,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     HepPlanner planner = createPlanner(program);
 
     planner.setRoot(root);
-    RelNode newRootRel = planner.findBestExp();
-
-    return newRootRel;
+    return planner.findBestExp();
   }
 
   protected RexNode decorrelateExpr(RexNode exp) {
@@ -312,9 +313,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
       RexNode exp,
       boolean projectPulledAboveLeftCorrelator) {
     RemoveCorrelationRexShuttle shuttle =
-        new RemoveCorrelationRexShuttle(
-            rexBuilder,
-            projectPulledAboveLeftCorrelator);
+        new RemoveCorrelationRexShuttle(rexBuilder,
+            projectPulledAboveLeftCorrelator, null, ImmutableSet.<Integer>of());
     return exp.accept(shuttle);
   }
 
@@ -323,10 +323,9 @@ public class RelDecorrelator implements ReflectiveVisitor {
       boolean projectPulledAboveLeftCorrelator,
       RexInputRef nullIndicator) {
     RemoveCorrelationRexShuttle shuttle =
-        new RemoveCorrelationRexShuttle(
-            rexBuilder,
-            projectPulledAboveLeftCorrelator,
-            nullIndicator);
+        new RemoveCorrelationRexShuttle(rexBuilder,
+            projectPulledAboveLeftCorrelator, nullIndicator,
+            ImmutableSet.<Integer>of());
     return exp.accept(shuttle);
   }
 
@@ -335,30 +334,27 @@ public class RelDecorrelator implements ReflectiveVisitor {
       boolean projectPulledAboveLeftCorrelator,
       Set<Integer> isCount) {
     RemoveCorrelationRexShuttle shuttle =
-        new RemoveCorrelationRexShuttle(
-            rexBuilder,
-            projectPulledAboveLeftCorrelator,
-            isCount);
+        new RemoveCorrelationRexShuttle(rexBuilder,
+            projectPulledAboveLeftCorrelator, null, isCount);
     return exp.accept(shuttle);
   }
 
-  public void decorrelateRelGeneric(RelNode rel) {
+  /** Fallback if none of the other {@code decorrelateRel} methods match. */
+  public Frame decorrelateRel(RelNode rel) {
     RelNode newRel = rel.copy(rel.getTraitSet(), rel.getInputs());
 
     if (rel.getInputs().size() > 0) {
       List<RelNode> oldInputs = rel.getInputs();
       List<RelNode> newInputs = Lists.newArrayList();
       for (int i = 0; i < oldInputs.size(); ++i) {
-        RelNode newInputRel = mapOldToNewRel.get(oldInputs.get(i));
-        if ((newInputRel == null)
-            || mapNewRelToMapCorVarToOutputPos.containsKey(newInputRel)) {
-          // if child is not rewritten, or if it produces correlated
+        final Frame frame = getInvoke(oldInputs.get(i), rel);
+        if (frame == null || !frame.corVarOutputPos.isEmpty()) {
+          // if input is not rewritten, or if it produces correlated
           // variables, terminate rewrite
-          return;
-        } else {
-          newInputs.add(newInputRel);
-          newRel.replaceInput(i, newInputRel);
+          return null;
         }
+        newInputs.add(frame.r);
+        newRel.replaceInput(i, frame.r);
       }
 
       if (!Util.equalShallow(oldInputs, newInputs)) {
@@ -368,12 +364,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     // the output position should not change since there are no corVars
     // coming from below.
-    Map<Integer, Integer> mapOldToNewOutputPos = Maps.newHashMap();
-    for (int i = 0; i < rel.getRowType().getFieldCount(); i++) {
-      mapOldToNewOutputPos.put(i, i);
-    }
-    mapOldToNewRel.put(rel, newRel);
-    mapNewRelToMapOldToNewOutputPos.put(newRel, mapOldToNewOutputPos);
+    return register(rel, newRel, identityMap(rel.getRowType().getFieldCount()),
+        ImmutableSortedMap.<Correlation, Integer>of());
   }
 
   /**
@@ -381,7 +373,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
    *
    * @param rel Sort to be rewritten
    */
-  public void decorrelateRel(Sort rel) {
+  public Frame decorrelateRel(Sort rel) {
     //
     // Rewrite logic:
     //
@@ -397,33 +389,39 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // Its output does not change the input ordering, so there's no
     // need to call propagateExpr.
 
-    RelNode oldChildRel = rel.getInput();
-
-    RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
-    if (newChildRel == null) {
-      // If child has not been rewritten, do not rewrite this rel.
-      return;
+    final RelNode oldInput = rel.getInput();
+    final Frame frame = getInvoke(oldInput, rel);
+    if (frame == null) {
+      // If input has not been rewritten, do not rewrite this rel.
+      return null;
     }
+    final RelNode newInput = frame.r;
 
-    Map<Integer, Integer> childMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newChildRel);
-    assert childMapOldToNewOutputPos != null;
     Mappings.TargetMapping mapping =
         Mappings.target(
-            childMapOldToNewOutputPos,
-            oldChildRel.getRowType().getFieldCount(),
-            newChildRel.getRowType().getFieldCount());
+            frame.oldToNewOutputPos,
+            oldInput.getRowType().getFieldCount(),
+            newInput.getRowType().getFieldCount());
 
     RelCollation oldCollation = rel.getCollation();
     RelCollation newCollation = RexUtil.apply(mapping, oldCollation);
 
-    final Sort newRel =
-        LogicalSort.create(newChildRel, newCollation, rel.offset, rel.fetch);
-
-    mapOldToNewRel.put(rel, newRel);
+    final Sort newSort =
+        LogicalSort.create(newInput, newCollation, rel.offset, rel.fetch);
 
     // Sort does not change input ordering
-    mapNewRelToMapOldToNewOutputPos.put(newRel, childMapOldToNewOutputPos);
+    return register(rel, newSort, frame.oldToNewOutputPos,
+        frame.corVarOutputPos);
+  }
+
+  /**
+   * Rewrites a {@link Values}.
+   *
+   * @param rel Values to be rewritten
+   */
+  public Frame decorrelateRel(Values rel) {
+    // There are no inputs, so rel does not need to be changed.
+    return null;
   }
 
   /**
@@ -431,7 +429,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
    *
    * @param rel Aggregate to rewrite
    */
-  public void decorrelateRel(LogicalAggregate rel) {
+  public Frame decorrelateRel(LogicalAggregate rel) {
     if (rel.getGroupType() != Aggregate.Group.SIMPLE) {
       throw new AssertionError(Bug.CALCITE_461_FIXED);
     }
@@ -439,7 +437,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // Rewrite logic:
     //
     // 1. Permute the group by keys to the front.
-    // 2. If the child of an aggregate produces correlated variables,
+    // 2. If the input of an aggregate produces correlated variables,
     //    add them to the group list.
     // 3. Change aggCalls to reference the new project.
     //
@@ -447,117 +445,107 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // Aggregate itself should not reference cor vars.
     assert !cm.mapRefRelToCorVar.containsKey(rel);
 
-    RelNode oldChildRel = rel.getInput();
-
-    RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
-    if (newChildRel == null) {
-      // If child has not been rewritten, do not rewrite this rel.
-      return;
+    final RelNode oldInput = rel.getInput();
+    final Frame frame = getInvoke(oldInput, rel);
+    if (frame == null) {
+      // If input has not been rewritten, do not rewrite this rel.
+      return null;
     }
+    assert !frame.corVarOutputPos.isEmpty();
+    final RelNode newInput = frame.r;
 
-    Map<Integer, Integer> childMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newChildRel);
-    assert childMapOldToNewOutputPos != null;
-
-    // map from newChildRel
-    Map<Integer, Integer> mapNewChildToProjOutputPos = Maps.newHashMap();
+    // map from newInput
+    Map<Integer, Integer> mapNewInputToProjOutputPos = Maps.newHashMap();
     final int oldGroupKeyCount = rel.getGroupSet().cardinality();
 
-    // LogicalProject projects the original expressions,
-    // plus any correlated variables the child wants to pass along.
+    // Project projects the original expressions,
+    // plus any correlated variables the input wants to pass along.
     final List<Pair<RexNode, String>> projects = Lists.newArrayList();
 
-    List<RelDataTypeField> newChildOutput =
-        newChildRel.getRowType().getFieldList();
+    List<RelDataTypeField> newInputOutput =
+        newInput.getRowType().getFieldList();
 
-    int newPos;
+    int newPos = 0;
 
-    // oldChildRel has the original group by keys in the front.
-    for (newPos = 0; newPos < oldGroupKeyCount; newPos++) {
-      int newChildPos = childMapOldToNewOutputPos.get(newPos);
-      projects.add(RexInputRef.of2(newChildPos, newChildOutput));
-      mapNewChildToProjOutputPos.put(newChildPos, newPos);
+    // oldInput has the original group by keys in the front.
+    final NavigableMap<Integer, RexLiteral> omittedConstants = new TreeMap<>();
+    for (int i = 0; i < oldGroupKeyCount; i++) {
+      final RexLiteral constant = projectedLiteral(newInput, i);
+      if (constant != null) {
+        // Exclude constants. Aggregate({true}) occurs because Aggregate({})
+        // would generate 1 row even when applied to an empty table.
+        omittedConstants.put(i, constant);
+        continue;
+      }
+      int newInputPos = frame.oldToNewOutputPos.get(i);
+      projects.add(RexInputRef.of2(newInputPos, newInputOutput));
+      mapNewInputToProjOutputPos.put(newInputPos, newPos);
+      newPos++;
     }
 
-    SortedMap<Correlation, Integer> mapCorVarToOutputPos = Maps.newTreeMap();
-
-    boolean produceCorVar =
-        mapNewRelToMapCorVarToOutputPos.containsKey(newChildRel);
-    if (produceCorVar) {
-      // If child produces correlated variables, move them to the front,
-      // right after any existing groupby fields.
+    final SortedMap<Correlation, Integer> mapCorVarToOutputPos = new TreeMap<>();
+    if (!frame.corVarOutputPos.isEmpty()) {
+      // If input produces correlated variables, move them to the front,
+      // right after any existing GROUP BY fields.
 
-      SortedMap<Correlation, Integer> childMapCorVarToOutputPos =
-          mapNewRelToMapCorVarToOutputPos.get(newChildRel);
-
-      // Now add the corVars from the child, starting from
+      // Now add the corVars from the input, starting from
       // position oldGroupKeyCount.
-      for (Correlation corVar
-          : childMapCorVarToOutputPos.keySet()) {
-        int newChildPos = childMapCorVarToOutputPos.get(corVar);
-        projects.add(RexInputRef.of2(newChildPos, newChildOutput));
+      for (Map.Entry<Correlation, Integer> entry
+          : frame.corVarOutputPos.entrySet()) {
+        projects.add(RexInputRef.of2(entry.getValue(), newInputOutput));
 
-        mapCorVarToOutputPos.put(corVar, newPos);
-        mapNewChildToProjOutputPos.put(newChildPos, newPos);
+        mapCorVarToOutputPos.put(entry.getKey(), newPos);
+        mapNewInputToProjOutputPos.put(entry.getValue(), newPos);
         newPos++;
       }
     }
 
     // add the remaining fields
     final int newGroupKeyCount = newPos;
-    for (int i = 0; i < newChildOutput.size(); i++) {
-      if (!mapNewChildToProjOutputPos.containsKey(i)) {
-        projects.add(RexInputRef.of2(i, newChildOutput));
-        mapNewChildToProjOutputPos.put(i, newPos);
+    for (int i = 0; i < newInputOutput.size(); i++) {
+      if (!mapNewInputToProjOutputPos.containsKey(i)) {
+        projects.add(RexInputRef.of2(i, newInputOutput));
+        mapNewInputToProjOutputPos.put(i, newPos);
         newPos++;
       }
     }
 
-    assert newPos == newChildOutput.size();
+    assert newPos == newInputOutput.size();
 
-    // This LogicalProject will be what the old child maps to,
-    // replacing any previous mapping from old child).
-    RelNode newProjectRel =
-        RelOptUtil.createProject(newChildRel, projects, false);
+    // This Project will be what the old input maps to,
+    // replacing any previous mapping from old input).
+    RelNode newProject =
+        RelOptUtil.createProject(newInput, projects, false);
 
     // update mappings:
-    // oldChildRel ----> newChildRel
+    // oldInput ----> newInput
     //
-    //                   newProjectRel
-    //                        |
-    // oldChildRel ---->  newChildRel
+    //                newProject
+    //                   |
+    // oldInput ----> newInput
     //
     // is transformed to
     //
-    // oldChildRel ----> newProjectRel
-    //                        |
-    //                   newChildRel
+    // oldInput ----> newProject
+    //                   |
+    //                newInput
     Map<Integer, Integer> combinedMap = Maps.newHashMap();
 
-    for (Integer oldChildPos : childMapOldToNewOutputPos.keySet()) {
-      combinedMap.put(
-          oldChildPos,
-          mapNewChildToProjOutputPos.get(
-              childMapOldToNewOutputPos.get(oldChildPos)));
+    for (Integer oldInputPos : frame.oldToNewOutputPos.keySet()) {
+      combinedMap.put(oldInputPos,
+          mapNewInputToProjOutputPos.get(
+              frame.oldToNewOutputPos.get(oldInputPos)));
     }
 
-    mapOldToNewRel.put(oldChildRel, newProjectRel);
-    mapNewRelToMapOldToNewOutputPos.put(newProjectRel, combinedMap);
+    register(oldInput, newProject, combinedMap, mapCorVarToOutputPos);
 
-    if (produceCorVar) {
-      mapNewRelToMapCorVarToOutputPos.put(
-          newProjectRel,
-          mapCorVarToOutputPos);
-    }
-
-    // now it's time to rewrite LogicalAggregate
+    // now it's time to rewrite the Aggregate
+    final ImmutableBitSet newGroupSet = ImmutableBitSet.range(newGroupKeyCount);
     List<AggregateCall> newAggCalls = Lists.newArrayList();
     List<AggregateCall> oldAggCalls = rel.getAggCallList();
 
-    // LogicalAggregate.Call oldAggCall;
-    int oldChildOutputFieldCount = oldChildRel.getRowType().getFieldCount();
-    int newChildOutputFieldCount =
-        newProjectRel.getRowType().getFieldCount();
+    int oldInputOutputFieldCount = rel.getGroupSet().cardinality();
+    int newInputOutputFieldCount = newGroupSet.cardinality();
 
     int i = -1;
     for (AggregateCall oldAggCall : oldAggCalls) {
@@ -567,7 +555,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
       List<Integer> aggArgs = Lists.newArrayList();
 
       // Adjust the aggregator argument positions.
-      // Note aggregator does not change input ordering, so the child
+      // Note aggregator does not change input ordering, so the input
       // output position mapping can be used to derive the new positions
       // for the argument.
       for (int oldPos : oldAggArgs) {
@@ -577,34 +565,57 @@ public class RelDecorrelator implements ReflectiveVisitor {
           : combinedMap.get(oldAggCall.filterArg);
 
       newAggCalls.add(
-          oldAggCall.adaptTo(newProjectRel, aggArgs, filterArg,
+          oldAggCall.adaptTo(newProject, aggArgs, filterArg,
               oldGroupKeyCount, newGroupKeyCount));
 
       // The old to new output position mapping will be the same as that
-      // of newProjectRel, plus any aggregates that the oldAgg produces.
+      // of newProject, plus any aggregates that the oldAgg produces.
       combinedMap.put(
-          oldChildOutputFieldCount + i,
-          newChildOutputFieldCount + i);
+          oldInputOutputFieldCount + i,
+          newInputOutputFieldCount + i);
     }
 
-    LogicalAggregate newAggregate =
-        LogicalAggregate.create(newProjectRel,
+    relBuilder.push(
+        LogicalAggregate.create(newProject,
             false,
-            ImmutableBitSet.range(newGroupKeyCount),
+            newGroupSet,
             null,
-            newAggCalls);
+            newAggCalls));
+
+    if (!omittedConstants.isEmpty()) {
+      final List<RexNode> postProjects = new ArrayList<>(relBuilder.fields());
+      for (Map.Entry<Integer, RexLiteral> entry
+          : omittedConstants.descendingMap().entrySet()) {
+        postProjects.add(entry.getKey() + frame.corVarOutputPos.size(),
+            entry.getValue());
+      }
+      relBuilder.project(postProjects);
+    }
 
-    mapOldToNewRel.put(rel, newAggregate);
+    // Aggregate does not change input ordering so corVars will be
+    // located at the same position as the input newProject.
+    return register(rel, relBuilder.build(), combinedMap, mapCorVarToOutputPos);
+  }
 
-    mapNewRelToMapOldToNewOutputPos.put(newAggregate, combinedMap);
+  public Frame getInvoke(RelNode r, RelNode parent) {
+    final Frame frame = dispatcher.invoke(r);
+    if (frame != null) {
+      map.put(r, frame);
+    }
+    currentRel = parent;
+    return frame;
+  }
 
-    if (produceCorVar) {
-      // LogicalAggregate does not change input ordering so corVars will be
-      // located at the same position as the input newProjectRel.
-      mapNewRelToMapCorVarToOutputPos.put(
-          newAggregate,
-          mapCorVarToOutputPos);
+  /** Returns a literal output field, or null if it is not literal. */
+  private static RexLiteral projectedLiteral(RelNode rel, int i) {
+    if (rel instanceof Project) {
+      final Project project = (Project) rel;
+      final RexNode node = project.getProjects().get(i);
+      if (node instanceof RexLiteral) {
+        return (RexLiteral) node;
+      }
     }
+    return null;
   }
 
   /**
@@ -612,34 +623,24 @@ public class RelDecorrelator implements ReflectiveVisitor {
    *
    * @param rel the project rel to rewrite
    */
-  public void decorrelateRel(LogicalProject rel) {
+  public Frame decorrelateRel(LogicalProject rel) {
     //
     // Rewrite logic:
     //
-    // 1. Pass along any correlated variables coming from the child.
+    // 1. Pass along any correlated variables coming from the input.
     //
 
-    RelNode oldChildRel = rel.getInput();
-
-    RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
-    if (newChildRel == null) {
-      // If child has not been rewritten, do not rewrite this rel.
-      return;
+    final RelNode oldInput = rel.getInput();
+    Frame frame = getInvoke(oldInput, rel);
+    if (frame == null) {
+      // If input has not been rewritten, do not rewrite this rel.
+      return null;
     }
-    List<RexNode> oldProj = rel.getProjects();
-    List<RelDataTypeField> relOutput = rel.getRowType().getFieldList();
-
-    Map<Integer, Integer> childMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newChildRel);
-    assert childMapOldToNewOutputPos != null;
-
-    Map<Integer, Integer> mapOldToNewOutputPos = Maps.newHashMap();
-
-    boolean produceCorVar =
-        mapNewRelToMapCorVarToOutputPos.containsKey(newChildRel);
+    final List<RexNode> oldProjects = rel.getProjects();
+    final List<RelDataTypeField> relOutput = rel.getRowType().getFieldList();
 
     // LogicalProject projects the original expressions,
-    // plus any correlated variables the child wants to pass along.
+    // plus any correlated variables the input wants to pass along.
     final List<Pair<RexNode, String>> projects = Lists.newArrayList();
 
     // If this LogicalProject has correlated reference, create value generator
@@ -647,55 +648,38 @@ public class RelDecorrelator implements ReflectiveVisitor {
     if (cm.mapRefRelToCorVar.containsKey(rel)) {
       decorrelateInputWithValueGenerator(rel);
 
-      // The old child should be mapped to the LogicalJoin created by
+      // The old input should be mapped to the LogicalJoin created by
       // rewriteInputWithValueGenerator().
-      newChildRel = mapOldToNewRel.get(oldChildRel);
-      produceCorVar = true;
+      frame = map.get(oldInput);
     }
 
     // LogicalProject projects the original expressions
+    final Map<Integer, Integer> mapOldToNewOutputPos = Maps.newHashMap();
     int newPos;
-    for (newPos = 0; newPos < oldProj.size(); newPos++) {
+    for (newPos = 0; newPos < oldProjects.size(); newPos++) {
       projects.add(
           newPos,
           Pair.of(
-              decorrelateExpr(oldProj.get(newPos)),
+              decorrelateExpr(oldProjects.get(newPos)),
               relOutput.get(newPos).getName()));
       mapOldToNewOutputPos.put(newPos, newPos);
     }
 
-    SortedMap<Correlation, Integer> mapCorVarToOutputPos = Maps.newTreeMap();
-
-    // Project any correlated variables the child wants to pass along.
-    if (produceCorVar) {
-      SortedMap<Correlation, Integer> childMapCorVarToOutputPos =
-          mapNewRelToMapCorVarToOutputPos.get(newChildRel);
-
-      // propagate cor vars from the new child
-      List<RelDataTypeField> newChildOutput =
-          newChildRel.getRowType().getFieldList();
-      for (Correlation corVar
-          : childMapCorVarToOutputPos.keySet()) {
-        int corVarPos = childMapCorVarToOutputPos.get(corVar);
-        projects.add(RexInputRef.of2(corVarPos, newChildOutput));
-        mapCorVarToOutputPos.put(corVar, newPos);
-        newPos++;
-      }
+    // Project any correlated variables the input wants to pass along.
+    final SortedMap<Correlation, Integer> mapCorVarToOutputPos = new TreeMap<>();
+    for (Map.Entry<Correlation, Integer> entry : frame.corVarOutputPos.entrySet()) {
+      projects.add(
+          RexInputRef.of2(entry.getValue(),
+              frame.r.getRowType().getFieldList()));
+      mapCorVarToOutputPos.put(entry.getKey(), newPos);
+      newPos++;
     }
 
-    RelNode newProjectRel =
-        RelOptUtil.createProject(newChildRel, projects, false);
-
-    mapOldToNewRel.put(rel, newProjectRel);
-    mapNewRelToMapOldToNewOutputPos.put(
-        newProjectRel,
-        mapOldToNewOutputPos);
+    RelNode newProject =
+        RelOptUtil.createProject(frame.r, projects, false);
 
-    if (produceCorVar) {
-      mapNewRelToMapCorVarToOutputPos.put(
-          newProjectRel,
-          mapCorVarToOutputPos);
-    }
+    return register(rel, newProject, mapOldToNewOutputPos,
+        mapCorVarToOutputPos);
   }
 
   /**
@@ -712,44 +696,37 @@ public class RelDecorrelator implements ReflectiveVisitor {
       Iterable<Correlation> correlations,
       int valueGenFieldOffset,
       SortedMap<Correlation, Integer> mapCorVarToOutputPos) {
-    RelNode resultRel = null;
+    final Map<RelNode, List<Integer>> mapNewInputToOutputPos =
+        new HashMap<>();
 
-    Map<RelNode, List<Integer>> mapNewInputRelToOutputPos = Maps.newHashMap();
-
-    Map<RelNode, Integer> mapNewInputRelToNewOffset = Maps.newHashMap();
-
-    RelNode oldInputRel;
-    RelNode newInputRel;
-    List<Integer> newLocalOutputPosList;
+    final Map<RelNode, Integer> mapNewInputToNewOffset = new HashMap<>();
 
     // inputRel provides the definition of a correlated variable.
     // Add to map all the referenced positions(relative to each input rel)
     for (Correlation corVar : correlations) {
-      int oldCorVarOffset = corVar.field;
+      final int oldCorVarOffset = corVar.field;
 
-      oldInputRel = cm.mapCorVarToCorRel.get(corVar.corr).getInput(0);
-      assert oldInputRel != null;
-      newInputRel = mapOldToNewRel.get(oldInputRel);
-      assert newInputRel != null;
+      final RelNode oldInput = getCorRel(corVar);
+      assert oldInput != null;
+      final Frame frame = map.get(oldInput);
+      assert frame != null;
+      final RelNode newInput = frame.r;
 
-      if (!mapNewInputRelToOutputPos.containsKey(newInputRel)) {
+      final List<Integer> newLocalOutputPosList;
+      if (!mapNewInputToOutputPos.containsKey(newInput)) {
         newLocalOutputPosList = Lists.newArrayList();
       } else {
         newLocalOutputPosList =
-            mapNewInputRelToOutputPos.get(newInputRel);
+            mapNewInputToOutputPos.get(newInput);
       }
 
-      Map<Integer, Integer> mapOldToNewOutputPos =
-          mapNewRelToMapOldToNewOutputPos.get(newInputRel);
-      assert mapOldToNewOutputPos != null;
-
-      int newCorVarOffset = mapOldToNewOutputPos.get(oldCorVarOffset);
+      final int newCorVarOffset = frame.oldToNewOutputPos.get(oldCorVarOffset);
 
       // Add all unique positions referenced.
       if (!newLocalOutputPosList.contains(newCorVarOffset)) {
         newLocalOutputPosList.add(newCorVarOffset);
       }
-      mapNewInputRelToOutputPos.put(newInputRel, newLocalOutputPosList);
+      mapNewInputToOutputPos.put(newInput, newLocalOutputPosList);
     }
 
     int offset = 0;
@@ -759,33 +736,34 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // To make sure the plan does not change in terms of join order,
     // join these rels based on their occurrence in cor var list which
     // is sorted.
-    Set<RelNode> joinedInputRelSet = Sets.newHashSet();
+    final Set<RelNode> joinedInputRelSet = Sets.newHashSet();
 
+    RelNode r = null;
     for (Correlation corVar : correlations) {
-      oldInputRel = cm.mapCorVarToCorRel.get(corVar.corr).getInput(0);
-      assert oldInputRel != null;
-      newInputRel = mapOldToNewRel.get(oldInputRel);
-      assert newInputRel != null;
+      final RelNode oldInput = getCorRel(corVar);
+      assert oldInput != null;
+      final RelNode newInput = map.get(oldInput).r;
+      assert newInput != null;
 
-      if (!joinedInputRelSet.contains(newInputRel)) {
-        RelNode projectRel =
+      if (!joinedInputRelSet.contains(newInput)) {
+        RelNode project =
             RelOptUtil.createProject(
-                newInputRel,
-                mapNewInputRelToOutputPos.get(newInputRel));
-        RelNode distinctRel = RelOptUtil.createDistinctRel(projectRel);
-        RelOptCluster cluster = distinctRel.getCluster();
+                newInput,
+                mapNewInputToOutputPos.get(newInput));
+        RelNode distinct = RelOptUtil.createDistinctRel(project);
+        RelOptCluster cluster = distinct.getCluster();
 
-        joinedInputRelSet.add(newInputRel);
-        mapNewInputRelToNewOffset.put(newInputRel, offset);
-        offset += distinctRel.getRowType().getFieldCount();
+        joinedInputRelSet.add(newInput);
+        mapNewInputToNewOffset.put(newInput, offset);
+        offset += distinct.getRowType().getFieldCount();
 
-        if (resultRel == null) {
-          resultRel = distinctRel;
+        if (r == null) {
+          r = distinct;
         } else {
-          resultRel =
-              LogicalJoin.create(resultRel, distinctRel,
+          r =
+              LogicalJoin.create(r, distinct,
                   cluster.getRexBuilder().makeLiteral(true),
-                  JoinRelType.INNER, ImmutableSet.<String>of());
+                  ImmutableSet.<CorrelationId>of(), JoinRelType.INNER);
         }
       }
     }
@@ -794,27 +772,26 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // the join output, leaving room for valueGenFieldOffset because
     // valueGenerators are joined with the original left input of the rel
     // referencing correlated variables.
-    int newOutputPos;
-    int newLocalOutputPos;
     for (Correlation corVar : correlations) {
-      // The first child of a correlatorRel is always the rel defining
+      // The first input of a Correlator is always the rel defining
       // the correlated variables.
-      newInputRel =
-          mapOldToNewRel.get(cm.mapCorVarToCorRel.get(corVar.corr).getInput(0));
-      newLocalOutputPosList = mapNewInputRelToOutputPos.get(newInputRel);
+      final RelNode oldInput = getCorRel(corVar);
+      assert oldInput != null;
+      final Frame frame = map.get(oldInput);
+      final RelNode newInput = frame.r;
+      assert newInput != null;
 
-      Map<Integer, Integer> mapOldToNewOutputPos =
-          mapNewRelToMapOldToNewOutputPos.get(newInputRel);
-      assert mapOldToNewOutputPos != null;
+      final List<Integer> newLocalOutputPosList =
+          mapNewInputToOutputPos.get(newInput);
 
-      newLocalOutputPos = mapOldToNewOutputPos.get(corVar.field);
+      final int newLocalOutputPos = frame.oldToNewOutputPos.get(corVar.field);
 
       // newOutputPos is the index of the cor var in the referenced
       // position list plus the offset of referenced position list of
-      // each newInputRel.
-      newOutputPos =
+      // each newInput.
+      final int newOutputPos =
           newLocalOutputPosList.indexOf(newLocalOutputPos)
-              + mapNewInputRelToNewOffset.get(newInputRel)
+              + mapNewInputToNewOffset.get(newInput)
               + valueGenFieldOffset;
 
       if (mapCorVarToOutputPos.containsKey(corVar)) {
@@ -823,53 +800,47 @@ public class RelDecorrelator implements ReflectiveVisitor {
       mapCorVarToOutputPos.put(corVar, newOutputPos);
     }
 
-    return resultRel;
+    return r;
   }
 
-  private void decorrelateInputWithValueGenerator(
-      RelNode rel) {
-    // currently only handles one child input
-    assert rel.getInputs().size() == 1;
-    RelNode oldChildRel = rel.getInput(0);
-    RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
-
-    Map<Integer, Integer> childMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newChildRel);
-    assert childMapOldToNewOutputPos != null;
+  private RelNode getCorRel(Correlation corVar) {
+    final RelNode r = cm.mapCorVarToCorRel.get(corVar.corr);
+    RelNode r2 = r.getInput(0);
+    if (r2 instanceof Join) {
+      r2 = r2.getInput(0);
+    }
+    return r2;
+  }
 
-    SortedMap<Correlation, Integer> mapCorVarToOutputPos = Maps.newTreeMap();
+  private void decorrelateInputWithValueGenerator(RelNode rel) {
+    // currently only handles one input input
+    assert rel.getInputs().size() == 1;
+    RelNode oldInput = rel.getInput(0);
+    final Frame frame = map.get(oldInput);
 
-    if (mapNewRelToMapCorVarToOutputPos.containsKey(newChildRel)) {
-      mapCorVarToOutputPos.putAll(
-          mapNewRelToMapCorVarToOutputPos.get(newChildRel));
-    }
+    final SortedMap<Correlation, Integer> mapCorVarToOutputPos =
+        new TreeMap<>(frame.corVarOutputPos);
 
     final Collection<Correlation> corVarList = cm.mapRefRelToCorVar.get(rel);
 
-    RelNode newLeftChildRel = newChildRel;
-
-    int leftChildOutputCount = newLeftChildRel.getRowType().getFieldCount();
+    int leftInputOutputCount = frame.r.getRowType().getFieldCount();
 
     // can directly add positions into mapCorVarToOutputPos since join
-    // does not change the output ordering from the children.
-    RelNode valueGenRel =
+    // does not change the output ordering from the inputs.
+    RelNode valueGen =
         createValueGenerator(
             corVarList,
-            leftChildOutputCount,
+            leftInputOutputCount,
             mapCorVarToOutputPos);
 
-    final Set<String> variablesStopped = Collections.emptySet();
-    RelNode joinRel =
-        LogicalJoin.create(newLeftChildRel, valueGenRel,
-            rexBuilder.makeLiteral(true), JoinRelType.INNER, variablesStopped);
-
-    mapOldToNewRel.put(oldChildRel, joinRel);
-    mapNewRelToMapCorVarToOutputPos.put(joinRel, mapCorVarToOutputPos);
+    RelNode join =
+        LogicalJoin.create(frame.r, valueGen, rexBuilder.makeLiteral(true),
+            ImmutableSet.<CorrelationId>of(), JoinRelType.INNER);
 
     // LogicalJoin or LogicalFilter does not change the old input ordering. All
     // input fields from newLeftInput(i.e. the original input to the old
     // LogicalFilter) are in the output and in the same position.
-    mapNewRelToMapOldToNewOutputPos.put(joinRel, childMapOldToNewOutputPos);
+    register(oldInput, join, frame.oldToNewOutputPos, mapCorVarToOutputPos);
   }
 
   /**
@@ -877,7 +848,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
    *
    * @param rel the filter rel to rewrite
    */
-  public void decorrelateRel(LogicalFilter rel) {
+  public Frame decorrelateRel(LogicalFilter rel) {
     //
     // Rewrite logic:
     //
@@ -894,53 +865,36 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // rewrite the filter condition using new input.
     //
 
-    RelNode oldChildRel = rel.getInput();
-
-    RelNode newChildRel = mapOldToNewRel.get(oldChildRel);
-    if (newChildRel == null) {
-      // If child has not been rewritten, do not rewrite this rel.
-      return;
+    final RelNode oldInput = rel.getInput();
+    Frame frame = getInvoke(oldInput, rel);
+    if (frame == null) {
+      // If input has not been rewritten, do not rewrite this rel.
+      return null;
     }
 
-    Map<Integer, Integer> childMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newChildRel);
-    assert childMapOldToNewOutputPos != null;
-
-    boolean produceCorVar =
-        mapNewRelToMapCorVarToOutputPos.containsKey(newChildRel);
-
     // If this LogicalFilter has correlated reference, create value generator
     // and produce the correlated variables in the new output.
     if (cm.mapRefRelToCorVar.containsKey(rel)) {
       decorrelateInputWithValueGenerator(rel);
 
-      // The old child should be mapped to the newly created LogicalJoin by
+      // The old input should be mapped to the newly created LogicalJoin by
       // rewriteInputWithValueGenerator().
-      newChildRel = mapOldToNewRel.get(oldChildRel);
-      produceCorVar = true;
+      frame = map.get(oldInput);
     }
 
     // Replace the filter expression to reference output of the join
     // Map filter to the new filter over join
-    RelNode newFilterRel =
+    RelNode newFilter =
         RelOptUtil.createFilter(
-            newChildRel,
+            frame.r,
             decorrelateExpr(rel.getCondition()));
 
-    mapOldToNewRel.put(rel, newFilterRel);
-
     // Filter does not change the input ordering.
-    mapNewRelToMapOldToNewOutputPos.put(
-        newFilterRel,
-        childMapOldToNewOutputPos);
-
-    if (produceCorVar) {
-      // filter rel does not permute the input all corvars produced by
-      // filter will have the same output positions in the child rel.
-      mapNewRelToMapCorVarToOutputPos.put(
-          newFilterRel,
-          mapNewRelToMapCorVarToOutputPos.get(newChildRel));
-    }
+    // Filter rel does not permute the input.
+    // All corvars produced by filter will have the same output positions in the
+    // input rel.
+    return register(rel, newFilter, frame.oldToNewOutputPos,
+        frame.corVarOutputPos);
   }
 
   /**
@@ -948,7 +902,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
    *
    * @param rel Correlator
    */
-  public void decorrelateRel(LogicalCorrelate rel) {
+  public Frame decorrelateRel(LogicalCorrelate rel) {
     //
     // Rewrite logic:
     //
@@ -959,126 +913,93 @@ public class RelDecorrelator implements ReflectiveVisitor {
     //
 
     // the right input to Correlator should produce correlated variables
-    RelNode oldLeftRel = rel.getInputs().get(0);
-    RelNode oldRightRel = rel.getInputs().get(1);
+    final RelNode oldLeft = rel.getInput(0);
+    final RelNode oldRight = rel.getInput(1);
 
-    RelNode newLeftRel = mapOldToNewRel.get(oldLeftRel);
-    RelNode newRightRel = mapOldToNewRel.get(oldRightRel);
+    final Frame leftFrame = getInvoke(oldLeft, rel);
+    final Frame rightFrame = getInvoke(oldRight, rel);
 
-    if ((newLeftRel == null) || (newRightRel == null)) {
-      // If any child has not been rewritten, do not rewrite this rel.
-      return;
+    if (leftFrame == null || rightFrame == null) {
+      // If any input has not been rewritten, do not rewrite this rel.
+      return null;
     }
 
-    SortedMap<Correlation, Integer> rightChildMapCorVarToOutputPos =
-        mapNewRelToMapCorVarToOutputPos.get(newRightRel);
-
-    if (rightChildMapCorVarToOutputPos == null) {
-      return;
+    if (rightFrame.corVarOutputPos.isEmpty()) {
+      return null;
     }
 
-    Map<Integer, Integer> leftChildMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newLeftRel);
-    assert leftChildMapOldToNewOutputPos != null;
-
-    Map<Integer, Integer> rightChildMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newRightRel);
-
-    assert rightChildMapOldToNewOutputPos != null;
-
-    SortedMap<Correlation, Integer> mapCorVarToOutputPos =
-        rightChildMapCorVarToOutputPos;
-
     assert rel.getRequiredColumns().cardinality()
-        <= rightChildMapCorVarToOutputPos.keySet().size();
+        <= rightFrame.corVarOutputPos.keySet().size();
 
     // Change correlator rel into a join.
     // Join all the correlated variables produced by this correlator rel
     // with the values generated and propagated from the right input
-    RexNode condition = rexBuilder.makeLiteral(true);
+    final SortedMap<Correlation, Integer> corVarOutputPos =
+        new TreeMap<>(rightFrame.corVarOutputPos);
+    final List<RexNode> conditions = new ArrayList<>();
     final List<RelDataTypeField> newLeftOutput =
-        newLeftRel.getRowType().getFieldList();
+        leftFrame.r.getRowType().getFieldList();
     int newLeftFieldCount = newLeftOutput.size();
 
     final List<RelDataTypeField> newRightOutput =
-        newRightRel.getRowType().getFieldList();
+        rightFrame.r.getRowType().getFieldList();
 
-    int newLeftPos;
-    int newRightPos;
     for (Map.Entry<Correlation, Integer> rightOutputPos
-        : Lists.newArrayList(rightChildMapCorVarToOutputPos.entrySet())) {
-      Correlation corVar = rightOutputPos.getKey();
+        : Lists.newArrayList(corVarOutputPos.entrySet())) {
+      final Correlation corVar = rightOutputPos.getKey();
       if (!corVar.corr.equals(rel.getCorrelationId())) {
         continue;
       }
-      newLeftPos = leftChildMapOldToNewOutputPos.get(corVar.field);
-      newRightPos = rightChildMapCorVarToOutputPos.get(corVar);
-      RexNode equi =
-          rexBuilder.makeCall(
-              SqlStdOperatorTable.EQUALS,
+      final int newLeftPos = leftFrame.oldToNewOutputPos.get(corVar.field);
+      final int newRightPos = rightOutputPos.getValue();
+      conditions.add(
+          rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
               RexInputRef.of(newLeftPos, newLeftOutput),
-              new RexInputRef(
-                  newLeftFieldCount + newRightPos,
-                  newRightOutput.get(newRightPos).getType()));
-      if (condition == rexBuilder.makeLiteral(true)) {
-        condition = equi;
-      } else {
-        condition =
-            rexBuilder.makeCall(
-                SqlStdOperatorTable.AND,
-                condition,
-                equi);
-      }
+              new RexInputRef(newLeftFieldCount + newRightPos,
+                  newRightOutput.get(newRightPos).getType())));
 
       // remove this cor var from output position mapping
-      mapCorVarToOutputPos.remove(corVar);
+      corVarOutputPos.remove(corVar);
     }
 
     // Update the output position for the cor vars: only pass on the cor
     // vars that are not used in the join key.
-    for (Correlation corVar : mapCorVarToOutputPos.keySet()) {
-      int newPos = mapCorVarToOutputPos.get(corVar) + newLeftFieldCount;
-      mapCorVarToOutputPos.put(corVar, newPos);
+    for (Correlation corVar : corVarOutputPos.keySet()) {
+      int newPos = corVarOutputPos.get(corVar) + newLeftFieldCount;
+      corVarOutputPos.put(corVar, newPos);
     }
 
     // then add any cor var from the left input. Do not need to change
     // output positions.
-    if (mapNewRelToMapCorVarToOutputPos.containsKey(newLeftRel)) {
-      mapCorVarToOutputPos.putAll(
-          mapNewRelToMapCorVarToOutputPos.get(newLeftRel));
-    }
+    corVarOutputPos.putAll(leftFrame.corVarOutputPos);
 
     // Create the mapping between the output of the old correlation rel
     // and the new join rel
-    Map<Integer, Integer> mapOldToNewOutputPos = Maps.newHashMap();
+    final Map<Integer, Integer> mapOldToNewOutputPos = Maps.newHashMap();
 
-    int oldLeftFieldCount = oldLeftRel.getRowType().getFieldCount();
+    int oldLeftFieldCount = oldLeft.getRowType().getFieldCount();
 
-    int oldRightFieldCount = oldRightRel.getRowType().getFieldCount();
+    int oldRightFieldCount = oldRight.getRowType().getFieldCount();
     assert rel.getRowType().getFieldCount()
         == oldLeftFieldCount + oldRightFieldCount;
 
     // Left input positions are not changed.
-    mapOldToNewOutputPos.putAll(leftChildMapOldToNewOutputPos);
+    mapOldToNewOutputPos.putAll(leftFrame.oldToNewOutputPos);
 
     // Right input positions are shifted by newLeftFieldCount.
     for (int i = 0; i < oldRightFieldCount; i++) {
       mapOldToNewOutputPos.put(
           i + oldLeftFieldCount,
-          rightChildMapOldToNewOutputPos.get(i) + newLeftFieldCount);
+          rightFrame.oldToNewOutputPos.get(i) + newLeftFieldCount);
     }
 
-    final Set<String> variablesStopped = Collections.emptySet();
-    RelNode newRel =
-        LogicalJoin.create(newLeftRel, newRightRel, condition,
-            rel.getJoinType().toJoinType(), variablesStopped);
-
-    mapOldToNewRel.put(rel, newRel);
-    mapNewRelToMapOldToNewOutputPos.put(newRel, mapOldToNewOutputPos);
+    final RexNode condition =
+        RexUtil.composeConjunction(rexBuilder, conditions, false);
+    RelNode newJoin =
+        LogicalJoin.create(leftFrame.r, rightFrame.r, condition,
+            ImmutableSet.<CorrelationId>of(), rel.getJoinType().toJoinType());
 
-    if (!mapCorVarToOutputPos.isEmpty()) {
-      mapNewRelToMapCorVarToOutputPos.put(newRel, mapCorVarToOutputPos);
-    }
+    return register(rel, newJoin, mapOldToNewOutputPos, corVarOutputPos);
   }
 
   /**
@@ -1086,7 +1007,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
    *
    * @param rel LogicalJoin
    */
-  public void decorrelateRel(LogicalJoin rel) {
+  public Frame decorrelateRel(LogicalJoin rel) {
     //
     // Rewrite logic:
     //
@@ -1094,77 +1015,52 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // 2. map output positions and produce cor vars if any.
     //
 
-    RelNode oldLeftRel = rel.getInputs().get(0);
-    RelNode oldRightRel = rel.getInputs().get(1);
+    final RelNode oldLeft = rel.getInput(0);
+    final RelNode oldRight = rel.getInput(1);
 
-    RelNode newLeftRel = mapOldToNewRel.get(oldLeftRel);
-    RelNode newRightRel = mapOldToNewRel.get(oldRightRel);
+    final Frame leftFrame = getInvoke(oldLeft, rel);
+    final Frame rightFrame = getInvoke(oldRight, rel);
 
-    if ((newLeftRel == null) || (newRightRel == null)) {
-      // If any child has not been rewritten, do not rewrite this rel.
-      return;
+    if (leftFrame == null || rightFrame == null) {
+      // If any input has not been rewritten, do not rewrite this rel.
+      return null;
     }
 
-    Map<Integer, Integer> leftChildMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newLeftRel);
-    assert leftChildMapOldToNewOutputPos != null;
-
-    Map<Integer, Integer> rightChildMapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newRightRel);
-    assert rightChildMapOldToNewOutputPos != null;
-
-    SortedMap<Correlation, Integer> mapCorVarToOutputPos = Maps.newTreeMap();
-
-    final Set<String> variablesStopped = Collections.emptySet();
-    RelNode newRel =
-        LogicalJoin.create(newLeftRel, newRightRel,
-            decorrelateExpr(rel.getCondition()), rel.getJoinType(),
-            variablesStopped);
+    final RelNode newJoin =
+        LogicalJoin.create(leftFrame.r, rightFrame.r,
+            decorrelateExpr(rel.getCondition()),
+            ImmutableSet.<CorrelationId>of(), rel.getJoinType());
 
     // Create the mapping between the output of the old correlation rel
     // and the new join rel
     Map<Integer, Integer> mapOldToNewOutputPos = Maps.newHashMap();
 
-    int oldLeftFieldCount = oldLeftRel.getRowType().getFieldCount();
-    int newLeftFieldCount = newLeftRel.getRowType().getFieldCount();
+    int oldLeftFieldCount = oldLeft.getRowType().getFieldCount();
+    int newLeftFieldCount = leftFrame.r.getRowType().getFieldCount();
 
-    int oldRightFieldCount = oldRightRel.getRowType().getFieldCount();
+    int oldRightFieldCount = oldRight.getRowType().getFieldCount();
     assert rel.getRowType().getFieldCount()
         == oldLeftFieldCount + oldRightFieldCount;
 
     // Left input positions are not changed.
-    mapOldToNewOutputPos.putAll(leftChildMapOldToNewOutputPos);
+    mapOldToNewOutputPos.putAll(leftFrame.oldToNewOutputPos);
 
     // Right input positions are shifted by newLeftFieldCount.
     for (int i = 0; i < oldRightFieldCount; i++) {
-      mapOldToNewOutputPos.put(
-          i + oldLeftFieldCount,
-          rightChildMapOldToNewOutputPos.get(i) + newLeftFieldCount);
+      mapOldToNewOutputPos.put(i + oldLeftFieldCount,
+          rightFrame.oldToNewOutputPos.get(i) + newLeftFieldCount);
     }
 
-    if (mapNewRelToMapCorVarToOutputPos.containsKey(newLeftRel)) {
-      mapCorVarToOutputPos.putAll(
-          mapNewRelToMapCorVarToOutputPos.get(newLeftRel));
-    }
+    final SortedMap<Correlation, Integer> mapCorVarToOutputPos =
+        new TreeMap<>(leftFrame.corVarOutputPos);
 
     // Right input positions are shifted by newLeftFieldCount.
-    int oldRightPos;
-    if (mapNewRelToMapCorVarToOutputPos.containsKey(newRightRel)) {
-      SortedMap<Correlation, Integer> rightChildMapCorVarToOutputPos =
-          mapNewRelToMapCorVarToOutputPos.get(newRightRel);
-      for (Correlation corVar : rightChildMapCorVarToOutputPos.keySet()) {
-        oldRightPos = rightChildMapCorVarToOutputPos.get(corVar);
-        mapCorVarToOutputPos.put(
-            corVar,
-            oldRightPos + newLeftFieldCount);
-      }
-    }
-    mapOldToNewRel.put(rel, newRel);
-    mapNewRelToMapOldToNewOutputPos.put(newRel, mapOldToNewOutputPos);
-
-    if (!mapCorVarToOutputPos.isEmpty()) {
-      mapNewRelToMapCorVarToOutputPos.put(newRel, mapCorVarToOutputPos);
+    for (Map.Entry<Correlation, Integer> entry
+        : rightFrame.corVarOutputPos.entrySet()) {
+      mapCorVarToOutputPos.put(entry.getKey(),
+          entry.getValue() + newLeftFieldCount);
     }
+    return register(rel, newJoin, mapOldToNewOutputPos, mapCorVarToOutputPos);
   }
 
   private RexInputRef getNewForOldInputRef(RexInputRef oldInputRef) {
@@ -1175,61 +1071,57 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     // determine which input rel oldOrdinal references, and adjust
     // oldOrdinal to be relative to that input rel
-    List<RelNode> oldInputRels = currentRel.getInputs();
-    RelNode oldInputRel = null;
+    RelNode oldInput = null;
 
-    for (RelNode oldInputRel0 : oldInputRels) {
-      RelDataType oldInputType = oldInputRel0.getRowType();
+    for (RelNode oldInput0 : currentRel.getInputs()) {
+      RelDataType oldInputType = oldInput0.getRowType();
       int n = oldInputType.getFieldCount();
       if (oldOrdinal < n) {
-        oldInputRel = oldInputRel0;
+        oldInput = oldInput0;
         break;
       }
-      RelNode newInput = mapOldToNewRel.get(oldInputRel0);
+      RelNode newInput = map.get(oldInput0).r;
       newOrdinal += newInput.getRowType().getFieldCount();
       oldOrdinal -= n;
     }
 
-    assert oldInputRel != null;
+    assert oldInput != null;
 
-    RelNode newInputRel = mapOldToNewRel.get(oldInputRel);
-    assert newInputRel != null;
+    final Frame frame = map.get(oldInput);
+    assert frame != null;
 
-    // now oldOrdinal is relative to oldInputRel
+    // now oldOrdinal is relative to oldInput
     int oldLocalOrdinal = oldOrdinal;
 
-    // figure out the newLocalOrdinal, relative to the newInputRel.
+    // figure out the newLocalOrdinal, relative to the newInput.
     int newLocalOrdinal = oldLocalOrdinal;
 
-    Map<Integer, Integer> mapOldToNewOutputPos =
-        mapNewRelToMapOldToNewOutputPos.get(newInputRel);
-
-    if (mapOldToNewOutputPos != null) {
-      newLocalOrdinal = mapOldToNewOutputPos.get(oldLocalOrdinal);
+    if (!frame.oldToNewOutputPos.isEmpty()) {
+      newLocalOrdinal = frame.oldToNewOutputPos.get(oldLocalOrdinal);
     }
 
     newOrdinal += newLocalOrdinal;
 
     return new RexInputRef(newOrdinal,
-        newInputRel.getRowType().getFieldList().get(newLocalOrdinal).getType());
+        frame.r.getRowType().getFieldList().get(newLocalOrdinal).getType());
   }
 
   /**
-   * Pull projRel above the join from its RHS input. Enforce nullability
+   * Pulls project above the join from its RHS input. Enforces nullability
    * for join output.
    *
    * @param join          Join
-   * @param projRel          the original projRel as the RHS input of the join.
+   * @param project       Original project as the right-hand input of the join
    * @param nullIndicatorPos Position of null indicator
    * @return the subtree with the new LogicalProject at the root
    */
   private RelNode projectJoinOutputWithNullability(
       LogicalJoin join,
-      LogicalProject projRel,
+      LogicalProject project,
       int nullIndicatorPos) {
-    RelDataTypeFactory typeFactory = join.getCluster().getTypeFactory();
-    RelNode leftInputRel = join.getLeft();
-    JoinRelType joinType = join.getJoinType();
+    final RelDataTypeFactory typeFactory = join.getCluster().getTypeFactory();
+    final RelNode left = join.getLeft();
+    final JoinRelType joinType = join.getJoinType();
 
     RexInputRef nullIndicator =
         new RexInputRef(
@@ -1245,7 +1137,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // project everything from the LHS and then those from the original
     // projRel
     List<RelDataTypeField> leftInputFields =
-        leftInputRel.getRowType().getFieldList();
+        left.getRowType().getFieldList();
 
     for (int i = 0; i < leftInputFields.size(); i++) {
       newProjExprs.add(RexInputRef.of2(i, leftInputFields));
@@ -1257,7 +1149,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     boolean projectPulledAboveLeftCorrelator =
         joinType.generatesNullsOnRight();
 
-    for (Pair<RexNode, String> pair : projRel.getNamedProjects()) {
+    for (Pair<RexNode, String> pair : project.getNamedProjects()) {
       RexNode newProjExpr =
           removeCorrelationExpr(
               pair.left,
@@ -1267,36 +1159,33 @@ public class RelDecorrelator implements ReflectiveVisitor {
       newProjExprs.add(Pair.of(newProjExpr, pair.right));
     }
 
-    RelNode newProjRel =
-        RelOptUtil.createProject(join, newProjExprs, false);
-
-    return newProjRel;
+    return RelOptUtil.createProject(join, newProjExprs, false);
   }
 
   /**
-   * Pulls projRel above the joinRel from its RHS input. Enforces nullability
-   * for join output.
+   * Pulls a {@link Project} above a {@link Correlate} from its RHS input.
+   * Enforces nullability for join output.
    *
-   * @param corRel  Correlator
-   * @param projRel the original LogicalProject as the RHS input of the join
+   * @param correlate  Correlate
+   * @param project the original project as the RHS input of the join
    * @param isCount Positions which are calls to the <code>COUNT</code>
    *                aggregation function
    * @return the subtree with the new LogicalProject at the root
    */
   private RelNode aggregateCorrelatorOutput(
-      LogicalCorrelate corRel,
-      LogicalProject projRel,
+      Correlate correlate,
+      LogicalProject project,
       Set<Integer> isCount) {
-    RelNode leftInputRel = corRel.getLeft();
-    JoinRelType joinType = corRel.getJoinType().toJoinType();
+    final RelNode left = correlate.getLeft();
+    final JoinRelType joinType = correlate.getJoinType().toJoinType();
 
     // now create the new project
-    List<Pair<RexNode, String>> newProjects = Lists.newArrayList();
+    final List<Pair<RexNode, String>> newProjects = Lists.newArrayList();
 
-    // project everything from the LHS and then those from the original
-    // projRel
-    List<RelDataTypeField> leftInputFields =
-        leftInputRel.getRowType().getFieldList();
+    // Project everything from the LHS and then those from the original
+    // project
+    final List<RelDataTypeField> leftInputFields =
+        left.getRowType().getFieldList();
 
     for (int i = 0; i < leftInputFields.size(); i++) {
       newProjects.add(RexInputRef.of2(i, leftInputFields));
@@ -1308,7 +1197,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     boolean projectPulledAboveLeftCorrelator =
         joinType.generatesNullsOnRight();
 
-    for (Pair<RexNode, String> pair : projRel.getNamedProjects()) {
+    for (Pair<RexNode, String> pair : project.getNamedProjects()) {
       RexNode newProjExpr =
           removeCorrelationExpr(
               pair.left,
@@ -1317,22 +1206,22 @@ public class RelDecorrelator implements ReflectiveVisitor {
       newProjects.add(Pair.of(newProjExpr, pair.right));
     }
 
-    return RelOptUtil.createProject(corRel, newProjects, false);
+    return RelOptUtil.createProject(correlate, newProjects, false);
   }
 
   /**
    * Checks whether the correlations in projRel and filter are related to
    * the correlated variables provided by corRel.
    *
-   * @param corRel    Correlator
-   * @param projRel   The original Project as the RHS input of the join
+   * @param correlate    Correlate
+   * @param project   The original Project as the RHS input of the join
    * @param filter    Filter
    * @param correlatedJoinKeys Correlated join keys
    * @return true if filter and proj only references corVar provided by corRel
    */
   private boolean checkCorVars(
-      LogicalCorrelate corRel,
-      LogicalProject projRel,
+      LogicalCorrelate correlate,
+      LogicalProject project,
       LogicalFilter filter,
       List<RexFieldAccess> correlatedJoinKeys) {
     if (filter != null) {
@@ -1344,8 +1233,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           Sets.newHashSet(cm.mapRefRelToCorVar.get(filter));
 
       for (RexFieldAccess correlatedJoinKey : correlatedJoinKeys) {
-        corVarInFilter.remove(
-            cm.mapFieldAccessToCorVar.get(correlatedJoinKey));
+        corVarInFilter.remove(cm.mapFieldAccessToCorVar.get(correlatedJoinKey));
       }
 
       if (!corVarInFilter.isEmpty()) {
@@ -1357,18 +1245,18 @@ public class RelDecorrelator implements ReflectiveVisitor {
       corVarInFilter.addAll(cm.mapRefRelToCorVar.get(filter));
 
       for (Correlation corVar : corVarInFilter) {
-        if (cm.mapCorVarToCorRel.get(corVar.corr) != corRel) {
+        if (cm.mapCorVarToCorRel.get(corVar.corr) != correlate) {
           return false;
         }
       }
     }
 
-    // if projRel has any correlated reference, make sure they are also
-    // provided by the current corRel. They will be projected out of the LHS
-    // of the corRel.
-    if ((projRel != null) && cm.mapRefRelToCorVar.containsKey(projRel)) {
-      for (Correlation corVar : cm.mapRefRelToCorVar.get(projRel)) {
-        if (cm.mapCorVarToCorRel.get(corVar.corr) != corRel) {
+    // if project has any correlated reference, make sure they are also
+    // provided by the current correlate. They will be projected out of the LHS
+    // of the correlate.
+    if ((project != null) && cm.mapRefRelToCorVar.containsKey(project)) {
+      for (Correlation corVar : cm.mapRefRelToCorVar.get(project)) {
+        if (cm.mapCorVarToCorRel.get(corVar.corr) != correlate) {
           return false;
         }
       }
@@ -1380,26 +1268,26 @@ public class RelDecorrelator implements ReflectiveVisitor {
   /**
    * Remove correlated variables from the tree at root corRel
    *
-   * @param corRel Correlator
+   * @param correlate Correlator
    */
-  private void removeCorVarFromTree(LogicalCorrelate corRel) {
-    if (cm.mapCorVarToCorRel.get(corRel.getCorrelationId()) == corRel) {
-      cm.mapCorVarToCorRel.remove(corRel.getCorrelationId());
+  private void removeCorVarFromTree(LogicalCorrelate correlate) {
+    if (cm.mapCorVarToCorRel.get(correlate.getCorrelationId()) == correlate) {
+      cm.mapCorVarToCorRel.remove(correlate.getCorrelationId());
     }
   }
 
   /**
-   * Project all childRel output fields plus the additional expressions.
+   * Projects all {@code input} output fields plus the additional expressions.
    *
-   * @param childRel        Child relational expression
+   * @param input        Input relational expression
    * @param additionalExprs Additional expressions and names
    * @return the new LogicalProject
    */
   private RelNode createProjectWithAdditionalExprs(
-      RelNode childRel,
+      RelNode input,
       List<Pair<RexNode, String>> additionalExprs) {
     final List<RelDataTypeField> fieldList =
-        childRel.getRowType().getFieldList();
+        input.getRowType().getFieldList();
     List<Pair<RexNode, String>> projects = Lists.newArrayList();
     for (Ord<RelDataTypeField> field : Ord.zip(fieldList)) {
       projects.add(
@@ -1409,140 +1297,93 @@ public class RelDecorrelator implements ReflectiveVisitor {
               field.e.getName()));
     }
     projects.addAll(additionalExprs);
-    return RelOptUtil.createProject(childRel, projects, false);
+    return RelOptUtil.createProject(input, projects, false);
   }
 
-  //~ Inner Classes ----------------------------------------------------------
+  /* Returns an immutable map with the identity [0: 0, .., count-1: count-1]. */
+  static Map<Integer, Integer> identityMap(int count) {
+    ImmutableMap.Builder<Integer, Integer> builder = ImmutableMap.builder();
+    for (int i = 0; i < count; i++) {
+      builder.put(i, i);
+    }
+    return builder.build();
+  }
+
+  /** Registers a relational expression and the relational expression it became
+   * after decorrelation. */
+  Frame register(RelNode rel, RelNode newRel,
+      Map<Integer, Integer> oldToNewOutputPos,
+      SortedMap<Correlation, Integer> corVarToOutputPos) {
+    assert allLessThan(oldToNewOutputPos.keySet(),
+        newRel.getRowType().getFieldCount(), Litmus.THROW);
+    final Frame frame = new Frame(newRel, corVarToOutputPos, oldToNewOutputPos);
+    map.put(rel, frame);
+    return frame;
+  }
 
-  /** Visitor that decorrelates. */
-  private class DecorrelateRelVisitor extends RelVisitor {
-    private final ReflectiveVisitDispatcher<RelDecorrelator, RelNode>
-    dispatcher =
-        ReflectUtil.createDispatcher(
-            RelDecorrelator.class,
-            RelNode.class);
-
-    // implement RelVisitor
-    public void visit(RelNode p, int ordinal, RelNode parent) {
-      // rewrite children first  (from left to right)
-      super.visit(p, ordinal, parent);
-
-      currentRel = p;
-
-      final String visitMethodName = "decorrelateRel";
-      boolean found =
-          dispatcher.invokeVisitor(
-              RelDecorrelator.this,
-              currentRel,
-              visitMethodName);
-      setCurrent(null, null);
-
-      if (!found) {
-        decorrelateRelGeneric(p);
+  static boolean allLessThan(Collection<Integer> integers, int limit,
+      Litmus ret) {
+    for (int value : integers) {
+      if (value >= limit) {
+        return ret.fail("out of range; value: " + value + ", limit: " + limit);
       }
-      // else no rewrite will occur. This will terminate the bottom-up
-      // rewrite. If root node of a RelNode tree is not rewritten, the
-      // original tree will be returned. See decorrelate() method.
     }
+    return ret.succeed();
   }
 
+  private static RelNode stripHep(RelNode rel) {
+    if (rel instanceof HepRelVertex) {
+      HepRelVertex hepRelVertex = (HepRelVertex) rel;
+      rel = hepRelVertex.getCurrentRel();
+    }
+    return rel;
+  }
+
+  //~ Inner Classes ----------------------------------------------------------
+
   /** Shuttle that decorrelates. */
   private class DecorrelateRexShuttle extends RexShuttle {
-    // override RexShuttle
-    public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
-      int newInputRelOutputOffset = 0;
-      RelNode oldInputRel;
-      RelNode newInputRel;
-      Integer newInputPos;
-
-      List<RelNode> inputs = currentRel.getInputs();
-      for (int i = 0; i < inputs.size(); i++) {
-        oldInputRel = inputs.get(i);
-        newInputRel = mapOldToNewRel.get(oldInputRel);
-
-        if ((newInputRel != null)
-            && mapNewRelToMapCorVarToOutputPos.containsKey(newInputRel)) {
-          SortedMap<Correlation, Integer> childMapCorVarToOutputPos =
-              mapNewRelToMapCorVarToOutputPos.get(newInputRel);
-
-          if (childMapCorVarToOutputPos != null) {
-            // try to find in this input rel the position of cor var
-            Correlation corVar = cm.mapFieldAccessToCorVar.get(fieldAccess);
-
-            if (corVar != null) {
-              newInputPos = childMapCorVarToOutputPos.get(corVar);
-              if (newInputPos != null) {
-                // this input rel does produce the cor var
-                // referenced
-                newInputPos += newInputRelOutputOffset;
-
-                // fieldAccess is assumed to have the correct
-                // type info.
-                RexInputRef newInput =
-                    new RexInputRef(
-                        newInputPos,
-                        fieldAccess.getType());
-                return newInput;
-              }
+    @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+      int newInputOutputOffset = 0;
+      for (RelNode input : currentRel.getInputs()) {
+        final Frame frame = map.get(input);
+
+        if (frame != null) {
+          // try to find in this input rel the position of cor var
+          final Correlation corVar = cm.mapFieldAccessToCorVar.get(fieldAccess);
+
+          if (corVar != null) {
+            Integer newInputPos = frame.corVarOutputPos.get(corVar);
+            if (newInputPos != null) {
+              // This input rel does produce the cor var referenced.
+              // Assume fieldAccess has the correct type info.
+              return new RexInputRef(newInputPos + newInputOutputOffset,
+                  fieldAccess.getType());
             }
           }
 
           // this input rel does not produce the cor var needed
-          newInputRelOutputOffset +=
-              newInputRel.getRowType().getFieldCount();
+          newInputOutputOffset += frame.r.getRowType().getFieldCount();
         } else {
           // this input rel is not rewritten
-          newInputRelOutputOffset +=
-              oldInputRel.getRowType().getFieldCount();
+          newInputOutputOffset += input.getRowType().getFieldCount();
         }
       }
       return fieldAccess;
     }
 
-    // override RexShuttle
-    public RexNode visitInputRef(RexInputRef inputRef) {
-      RexInputRef newInputRef = getNewForOldInputRef(inputRef);
-      return newInputRef;
+    @Override public RexNode visitInputRef(RexInputRef inputRef) {
+      return getNewForOldInputRef(inputRef);
     }
   }
 
   /** Shuttle that removes correlations. */
   private class RemoveCorrelationRexShuttle extends RexShuttle {
-    RexBuilder rexBuilder;
-    RelDataTypeFactory typeFactory;
-    boolean projectPulledAboveLeftCorrelator;
-    RexInputRef nullIndicator;
-    Set<Integer> isCount;
-
-    public RemoveCorrelationRexShuttle(
-        RexBuilder rexBuilder,
-        boolean projectPulledAboveLeftCorrelator) {
-      this(
-          rexBuilder,
-          projectPulledAboveLeftCorrelator,
-          null, null);
-    }
-
-    public RemoveCorrelationRexShuttle(
-        RexBuilder rexBuilder,
-        boolean projectPulledAboveLeftCorrelator,
-        RexInputRef nullIndicator) {
-      this(
-          rexBuilder,
-          projectPulledAboveLeftCorrelator,
-          nullIndicator,
-          null);
-    }
-
-    public RemoveCorrelationRexShuttle(
-        RexBuilder rexBuilder,
-        boolean projectPulledAboveLeftCorrelator,
-        Set<Integer> isCount) {
-      this(
-          rexBuilder,
-          projectPulledAboveLeftCorrelator,
-          null, isCount);
-    }
+    final RexBuilder rexBuilder;
+    final RelDataTypeFactory typeFactory;
+    final boolean projectPulledAboveLeftCorrelator;
+    final RexInputRef nullIndicator;
+    final ImmutableSet<Integer> isCount;
 
     public RemoveCorrelationRexShuttle(
         RexBuilder rexBuilder,
@@ -1551,8 +1392,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
         Set<Integer> isCount) {
       this.projectPulledAboveLeftCorrelator =
           projectPulledAboveLeftCorrelator;
-      this.nullIndicator = nullIndicator;
-      this.isCount = isCount;
+      this.nullIndicator = nullIndicator; // may be null
+      this.isCount = ImmutableSet.copyOf(isCount);
       this.rexBuilder = rexBuilder;
       this.typeFactory = rexBuilder.getTypeFactory();
     }
@@ -1603,8 +1444,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
           caseOperands);
     }
 
-    // override RexShuttle
-    public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+    @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
       if (cm.mapFieldAccessToCorVar.containsKey(fieldAccess)) {
         // if it is a corVar, change it to be input ref.
         Correlation corVar = cm.mapFieldAccessToCorVar.get(fieldAccess);
@@ -1629,15 +1469,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
       return fieldAccess;
     }
 
-    // override RexShuttle
-    public RexNode visitInputRef(RexInputRef inputRef) {
-      if ((currentRel != null) && (currentRel instanceof LogicalCorrelate)) {
+    @Override public RexNode visitInputRef(RexInputRef inputRef) {
+      if (currentRel instanceof LogicalCorrelate) {
         // if this rel references corVar
         // and now it needs to be rewritten
         // it must have been pulled above the Correlator
         // replace the input ref to account for the LHS of the
         // Correlator
-        int leftInputFieldCount =
+        final int leftInputFieldCount =
             ((LogicalCorrelate) currentRel).getLeft().getRowType()
                 .getFieldCount();
         RelDataType newType = inputRef.getType();
@@ -1663,8 +1502,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
       return inputRef;
     }
 
-    // override RexLiteral
-    public RexNode visitLiteral(RexLiteral literal) {
+    @Override public RexNode visitLiteral(RexLiteral literal) {
       // Use nullIndicator to decide whether to project null.
       // Do nothing if the literal is null.
       if (!RexUtil.isNull(literal)
@@ -1678,7 +1516,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
       return literal;
     }
 
-    public RexNode visitCall(final RexCall call) {
+    @Override public RexNode visitCall(final RexCall call) {
       RexNode newCall;
 
       boolean[] update = {false};
@@ -1752,14 +1590,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     public void onMatch(RelOptRuleCall call) {
-      LogicalAggregate singleAggRel = call.rel(0);
-      LogicalProject projRel = call.rel(1);
-      LogicalAggregate aggRel = call.rel(2);
+      LogicalAggregate singleAggregate = call.rel(0);
+      LogicalProject project = call.rel(1);
+      LogicalAggregate aggregate = call.rel(2);
 
       // check singleAggRel is single_value agg
-      if ((!singleAggRel.getGroupSet().isEmpty())
-          || (singleAggRel.getAggCallList().size() != 1)
-          || !(singleAggRel.getAggCallList().get(0).getAggregation()
+      if ((!singleAggregate.getGroupSet().isEmpty())
+          || (singleAggregate.getAggCallList().size() != 1)
+          || !(singleAggregate.getAggCallList().get(0).getAggregation()
           instanceof SqlSingleValueAggFunction)) {
         return;
       }
@@ -1767,21 +1605,21 @@ public class RelDecorrelator implements ReflectiveVisitor {
       // check projRel only projects one expression
       // check this project only projects one expression, i.e. scalar
       // subqueries.
-      List<RexNode> projExprs = projRel.getProjects();
+      List<RexNode> projExprs = project.getProjects();
       if (projExprs.size() != 1) {
         return;
       }
 
       // check the input to projRel is an aggregate on the entire input
-      if (!aggRel.getGroupSet().isEmpty()) {
+      if (!aggregate.getGroupSet().isEmpty()) {
         return;
       }
 
       // singleAggRel produces a nullable type, so create the new
       // projection that casts proj expr to a nullable type.
-      final RelOptCluster cluster = projRel.getCluster();
-      RelNode newProjRel =
-          RelOptUtil.createProject(aggRel,
+      final RelOptCluster cluster = project.getCluster();
+      RelNode newProject =
+          RelOptUtil.createProject(aggregate,
               ImmutableList.of(
                   rexBuilder.makeCast(
                       cluster.getTypeFactory().createTypeWithNullability(
@@ -1789,7 +1627,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
                           true),
                       projExprs.get(0))),
               null);
-      call.transformTo(newProjRel);
+      call.transformTo(newProject);
     }
   }
 
@@ -1805,14 +1643,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     public void onMatch(RelOptRuleCall call) {
-      LogicalCorrelate corRel = call.rel(0);
-      RelNode leftInputRel = call.rel(1);
-      LogicalAggregate aggRel = call.rel(2);
-      LogicalProject projRel = call.rel(3);
-      RelNode rightInputRel = call.rel(4);
-      RelOptCluster cluster = corRel.getCluster();
+      final LogicalCorrelate correlate = call.rel(0);
+      final RelNode left = call.rel(1);
+      final LogicalAggregate aggregate = call.rel(2);
+      final LogicalProject project = call.rel(3);
+      RelNode right = call.rel(4);
+      final RelOptCluster cluster = correlate.getCluster();
 
-      setCurrent(call.getPlanner().getRoot(), corRel);
+      setCurrent(call.getPlanner().getRoot(), correlate);
 
       // Check for this pattern.
       // The pattern matching could be simplified if rules can be applied
@@ -1823,7 +1661,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
       //   LogicalAggregate (groupby (0) single_value())
       //     LogicalProject-A (may reference coVar)
       //       RightInputRel
-      JoinRelType joinType = corRel.getJoinType().toJoinType();
+      final JoinRelType joinType = correlate.getJoinType().toJoinType();
 
       // corRel.getCondition was here, however Correlate was updated so it
       // never includes a join condition. The code was not modified for brevity.
@@ -1835,23 +1673,23 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
       // check that the agg is of the following type:
       // doing a single_value() on the entire input
-      if ((!aggRel.getGroupSet().isEmpty())
-          || (aggRel.getAggCallList().size() != 1)
-          || !(aggRel.getAggCallList().get(0).getAggregation()
+      if ((!aggregate.getGroupSet().isEmpty())
+          || (aggregate.getAggCallList().size() != 1)
+          || !(aggregate.getAggCallList().get(0).getAggregation()
           instanceof SqlSingleValueAggFunction)) {
         return;
       }
 
       // check this project only projects one expression, i.e. scalar
       // subqueries.
-      if (projRel.getProjects().size() != 1) {
+      if (project.getProjects().size() != 1) {
         return;
       }
 
       int nullIndicatorPos;
 
-      if ((rightInputRel instanceof LogicalFilter)
-          && cm.mapRefRelToCorVar.containsKey(rightInputRel)) {
+      if ((right instanceof LogicalFilter)
+          && cm.mapRefRelToCorVar.containsKey(right)) {
         // rightInputRel has this shape:
         //
         //       LogicalFilter (references corvar)
@@ -1861,14 +1699,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
         // reference, make sure the correlated keys in the filter
         // condition forms a unique key of the RHS.
 
-        LogicalFilter filter = (LogicalFilter) rightInputRel;
-        rightInputRel = filter.getInput();
+        LogicalFilter filter = (LogicalFilter) right;
+        right = filter.getInput();
 
-        assert rightInputRel instanceof HepRelVertex;
-        rightInputRel = ((HepRelVertex) rightInputRel).getCurrentRel();
+        assert right instanceof HepRelVertex;
+        right = ((HepRelVertex) right).getCurrentRel();
 
         // check filter input contains no correlation
-        if (RelOptUtil.getVariablesUsed(rightInputRel).size() > 0) {
+        if (RelOptUtil.getVariablesUsed(right).size() > 0) {
           return;
         }
 
@@ -1889,7 +1727,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
         // check that the columns referenced in these comparisons form
         // an unique key of the filterInputRel
-        List<RexInputRef> rightJoinKeys = new ArrayList<RexInputRef>();
+        final List<RexInputRef> rightJoinKeys = new ArrayList<>();
         for (RexNode key : tmpRightJoinKeys) {
           assert key instanceof RexInputRef;
           rightJoinKeys.add((RexInputRef) key);
@@ -1904,11 +1742,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
         // The join filters out the nulls.  So, it's ok if there are
         // nulls in the join keys.
         if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
-            rightInputRel,
+            right,
             rightJoinKeys)) {
           SQL2REL_LOGGER.fine(rightJoinKeys.toString()
               + "are not unique keys for "
-              + rightInputRel.toString());
+              + right.toString());
           return;
         }
 
@@ -1918,7 +1756,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         List<RexFieldAccess> correlatedKeyList =
             visitor.getFieldAccessList();
 
-        if (!checkCorVars(corRel, projRel, filter, correlatedKeyList)) {
+        if (!checkCorVars(correlate, project, filter, correlatedKeyList)) {
           return;
         }
 
@@ -1935,15 +1773,15 @@ public class RelDecorrelator implements ReflectiveVisitor {
             removeCorrelationExpr(filter.getCondition(), false);
 
         nullIndicatorPos =
-            leftInputRel.getRowType().getFieldCount()
+            left.getRowType().getFieldCount()
                 + rightJoinKeys.get(0).getIndex();
-      } else if (cm.mapRefRelToCorVar.containsKey(projRel)) {
+      } else if (cm.mapRefRelToCorVar.containsKey(project)) {
         // check filter input contains no correlation
-        if (RelOptUtil.getVariablesUsed(rightInputRel).size() > 0) {
+        if (RelOptUtil.getVariablesUsed(right).size() > 0) {
           return;
         }
 
-        if (!checkCorVars(corRel, projRel, null, null)) {
+        if (!checkCorVars(correlate, project, null, null)) {
           return;
         }
 
@@ -1957,37 +1795,37 @@ public class RelDecorrelator implements ReflectiveVisitor {
         //         ProjInputRel
 
         // make the new projRel to provide a null indicator
-        rightInputRel =
-            createProjectWithAdditionalExprs(rightInputRel,
+        right =
+            createProjectWithAdditionalExprs(right,
                 ImmutableList.of(
                     Pair.<RexNode, String>of(
                         rexBuilder.makeLiteral(true), "nullIndicator")));
 
         // make the new aggRel
-        rightInputRel =
-            RelOptUtil.createSingleValueAggRel(cluster, rightInputRel);
+        right =
+            RelOptUtil.createSingleValueAggRel(cluster, right);
 
         // The last field:
         //     single_value(true)
         // is the nullIndicator
         nullIndicatorPos =
-            leftInputRel.getRowType().getFieldCount()
-                + rightInputRel.getRowType().getFieldCount() - 1;
+            left.getRowType().getFieldCount()
+                + right.getRowType().getFieldCount() - 1;
       } else {
         return;
       }
 
       // make the new join rel
       LogicalJoin join =
-          LogicalJoin.create(leftInputRel, rightInputRel, joinCond, joinType,
-              ImmutableSet.<String>of());
+          LogicalJoin.create(left, right, joinCond,
+              ImmutableSet.<CorrelationId>of(), joinType);
 
-      RelNode newProjRel =
-          projectJoinOutputWithNullability(join, projRel, nullIndicatorPos);
+      RelNode newProject =
+          projectJoinOutputWithNullability(join, project, nullIndicatorPos);
 
-      call.transformTo(newProjRel);
+      call.transformTo(newProject);
 
-      removeCorVarFromTree(corRel);
+      removeCorVarFromTree(correlate);
     }
   }
 
@@ -2005,15 +1843,15 @@ public class RelDecorrelator implements ReflectiveVisitor {
     }
 
     public void onMatch(RelOptRuleCall call) {
-      LogicalCorrelate corRel = call.rel(0);
-      RelNode leftInputRel = call.rel(1);
-      LogicalProject aggOutputProjRel = call.rel(2);
-      LogicalAggregate aggRel = call.rel(3);
-      LogicalProject aggInputProjRel = call.rel(4);
-      RelNode rightInputRel = call.rel(5);
-      RelOptCluster cluster = corRel.getCluster();
+      final LogicalCorrelate correlate = call.rel(0);
+      final RelNode left = call.rel(1);
+      final LogicalProject aggOutputProject = call.rel(2);
+      final LogicalAggregate aggregate = call.rel(3);
+      final LogicalProject aggInputProject = call.rel(4);
+      RelNode right = call.rel(5);
+      final RelOptCluster cluster = correlate.getCluster();
 
-      setCurrent(call.getPlanner().getRoot(), corRel);
+      setCurrent(call.getPlanner().getRoot(), correlate);
 
       // check for this pattern
       // The pattern matching could be simplified if rules can be applied
@@ -2026,13 +1864,13 @@ public class RelDecorrelator implements ReflectiveVisitor {
       //       LogicalProject-B (references coVar)
       //         rightInputRel
 
-      // check aggOutputProj projects only one expression
-      List<RexNode> aggOutputProjExprs = aggOutputProjRel.getProjects();
-      if (aggOutputProjExprs.size() != 1) {
+      // check aggOutputProject projects only one expression
+      final List<RexNode> aggOutputProjects = aggOutputProject.getProjects();
+      if (aggOutputProjects.size() != 1) {
         return;
       }
 
-      JoinRelType joinType = corRel.getJoinType().toJoinType();
+      final JoinRelType joinType = correlate.getJoinType().toJoinType();
       // corRel.getCondition was here, however Correlate was updated so it
       // never includes a join condition. The code was not modified for brevity.
       RexNode joinCond = rexBuilder.makeLiteral(true);
@@ -2042,14 +1880,14 @@ public class RelDecorrelator implements ReflectiveVisitor {
       }
 
       // check that the agg is on the entire input
-      if (!aggRel.getGroupSet().isEmpty()) {
+      if (!aggregate.getGroupSet().isEmpty()) {
         return;
       }
 
-      List<RexNode> aggInputProjExprs = aggInputProjRel.getProjects();
+      final List<RexNode> aggInputProjects = aggInputProject.getProjects();
 
-      List<AggregateCall> aggCalls = aggRel.getAggCallList();
-      Set<Integer> isCountStar = Sets.newHashSet();
+      final List<AggregateCall> aggCalls = aggregate.getAggCallList();
+      final Set<Integer> isCountStar = Sets.newHashSet();
 
       // mark if agg produces count(*) which needs to reference the
       // nullIndicator after the transformation.
@@ -2062,20 +1900,20 @@ public class RelDecorrelator implements ReflectiveVisitor {
         }
       }
 
-      if ((rightInputRel instanceof LogicalFilter)
-          && cm.mapRefRelToCorVar.containsKey(rightInputRel)) {
+      if ((right instanceof LogicalFilter)
+          && cm.mapRefRelToCorVar.containsKey(right)) {
         // rightInputRel has this shape:
         //
         //       LogicalFilter (references corvar)
         //         FilterInputRel
-        LogicalFilter filter = (LogicalFilter) rightInputRel;
-        rightInputRel = filter.getInput();
+        LogicalFilter filter = (LogicalFilter) right;
+        right = filter.getInput();
 
-        assert rightInputRel instanceof HepRelVertex;
-        rightInputRel = ((HepRelVertex) rightInputRel).getCurrentRel();
+        assert right instanceof HepRelVertex;
+        right = ((HepRelVertex) right).getCurrentRel();
 
         // check filter input contains no correlation
-        if (RelOptUtil.getVariablesUsed(rightInputRel).size() > 0) {
+        if (RelOptUtil.getVariablesUsed(right).size() > 0) {
           return;
         }
 
@@ -2119,17 +1957,17 @@ public class RelDecorrelator implements ReflectiveVisitor {
         // The join filters out the nulls.  So, it's ok if there are
         // nulls in the join keys.
         if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(
-            leftInputRel,
+            left,
             correlatedInputRefJoinKeys)) {
           SQL2REL_LOGGER.fine(correlatedJoinKeys.toString()
               + "are not unique keys for "
-              + leftInputRel.toString());
+              + left.toString());
           return;
         }
 
         // check cor var references are valid
-        if (!checkCorVars(corRel,
-            aggInputProjRel,
+        if (!checkCorVars(correlate,
+            aggInputProject,
             filter,
             correlatedJoinKeys)) {
           return;
@@ -2180,27 +2018,27 

<TRUNCATED>