You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jc...@apache.org on 2017/04/26 19:18:35 UTC

[01/11] calcite git commit: [CALCITE-1731] Materialized view rewriting for join and aggregate operators

Repository: calcite
Updated Branches:
  refs/heads/master 478de569c -> e9d0ca673


http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 dbc230a..af25b30 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -153,7 +153,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   private static final double DEPT_SIZE = 4d;
 
-  private static final String EMP_QNAME = "[CATALOG, SALES, EMP]";
+  private static final List<String> EMP_QNAME = ImmutableList.of("CATALOG", "SALES", "EMP");
 
   //~ Methods ----------------------------------------------------------------
 
@@ -1513,7 +1513,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final String inputRef = RexInputRef.of(4, tableRel.getRowType().getFieldList()).toString();
     assertThat(r.size(), is(1));
     final String resultString = r.iterator().next().toString();
-    assertThat(resultString, startsWith(EMP_QNAME));
+    assertThat(resultString, startsWith(EMP_QNAME.toString()));
     assertThat(resultString, endsWith(inputRef));
   }
 
@@ -1527,14 +1527,14 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final Set<RexNode> r1 = mq.getExpressionLineage(rel, ref1);
     assertThat(r1.size(), is(1));
     final RexTableInputRef result1 = (RexTableInputRef) r1.iterator().next();
-    assertThat(result1.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(result1.getQualifiedName().equals(EMP_QNAME));
     assertThat(result1.getIndex(), is(3));
 
     final RexNode ref2 = RexInputRef.of(1, rel.getRowType().getFieldList());
     final Set<RexNode> r2 = mq.getExpressionLineage(rel, ref2);
     assertThat(r2.size(), is(1));
     final RexTableInputRef result2 = (RexTableInputRef) r2.iterator().next();
-    assertThat(result2.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(result2.getQualifiedName().equals(EMP_QNAME));
     assertThat(result2.getIndex(), is(7));
 
     assertThat(result1.getIdentifier(), is(result2.getIdentifier()));
@@ -1550,14 +1550,14 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final Set<RexNode> r1 = mq.getExpressionLineage(rel, ref1);
     assertThat(r1.size(), is(1));
     final RexTableInputRef result1 = (RexTableInputRef) r1.iterator().next();
-    assertThat(result1.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(result1.getQualifiedName().equals(EMP_QNAME));
     assertThat(result1.getIndex(), is(7));
 
     final RexNode ref2 = RexInputRef.of(1, rel.getRowType().getFieldList());
     final Set<RexNode> r2 = mq.getExpressionLineage(rel, ref2);
     assertThat(r2.size(), is(1));
     final RexTableInputRef result2 = (RexTableInputRef) r2.iterator().next();
-    assertThat(result2.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(result2.getQualifiedName().equals(EMP_QNAME));
     assertThat(result2.getIndex(), is(3));
 
     assertThat(result1.getIdentifier(), is(result2.getIdentifier()));
@@ -1578,10 +1578,10 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final RexCall call = (RexCall) result;
     assertThat(call.getOperands().size(), is(2));
     final RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
-    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(inputRef1.getQualifiedName().equals(EMP_QNAME));
     assertThat(inputRef1.getIndex(), is(0));
     final RexTableInputRef inputRef2 = (RexTableInputRef) call.getOperands().get(1);
-    assertThat(inputRef2.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(inputRef2.getQualifiedName().equals(EMP_QNAME));
     assertThat(inputRef2.getIndex(), is(7));
     assertThat(inputRef1.getIdentifier(), is(inputRef2.getIdentifier()));
   }
@@ -1595,7 +1595,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
     assertThat(r.size(), is(1));
     final RexTableInputRef result = (RexTableInputRef) r.iterator().next();
-    assertThat(result.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(result.getQualifiedName().equals(EMP_QNAME));
     assertThat(result.getIndex(), is(1));
   }
 
@@ -1608,7 +1608,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
     assertThat(r.size(), is(1));
     final RexTableInputRef result = (RexTableInputRef) r.iterator().next();
-    assertThat(result.getQualifiedName(), is("[CATALOG, SALES, BONUS]"));
+    assertTrue(result.getQualifiedName().equals(ImmutableList.of("CATALOG", "SALES", "BONUS")));
     assertThat(result.getIndex(), is(0));
   }
 
@@ -1626,7 +1626,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final String inputRef1 = RexInputRef.of(7, tableRel.getRowType().getFieldList()).toString();
     assertThat(r1.size(), is(1));
     final String resultString1 = r1.iterator().next().toString();
-    assertThat(resultString1, startsWith(EMP_QNAME));
+    assertThat(resultString1, startsWith(EMP_QNAME.toString()));
     assertThat(resultString1, endsWith(inputRef1));
 
     final RexNode ref2 = RexInputRef.of(1, rel.getRowType().getFieldList());
@@ -1634,7 +1634,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final String inputRef2 = RexInputRef.of(5, tableRel.getRowType().getFieldList()).toString();
     assertThat(r2.size(), is(1));
     final String resultString2 = r2.iterator().next().toString();
-    assertThat(resultString2, startsWith(EMP_QNAME));
+    assertThat(resultString2, startsWith(EMP_QNAME.toString()));
     assertThat(resultString2, endsWith(inputRef2));
 
     assertThat(((RexTableInputRef) r1.iterator().next()).getIdentifier(),
@@ -1663,7 +1663,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final String inputRef = RexInputRef.of(1, tableRel.getRowType().getFieldList()).toString();
     assertThat(r.size(), is(1));
     final String resultString = r.iterator().next().toString();
-    assertThat(resultString, startsWith(EMP_QNAME));
+    assertThat(resultString, startsWith(EMP_QNAME.toString()));
     assertThat(resultString, endsWith(inputRef));
   }
 
@@ -1679,7 +1679,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final String inputRef = RexInputRef.of(7, tableRel.getRowType().getFieldList()).toString();
     assertThat(r.size(), is(1));
     final String resultString = r.iterator().next().toString();
-    assertThat(resultString, startsWith(EMP_QNAME));
+    assertThat(resultString, startsWith(EMP_QNAME.toString()));
     assertThat(resultString, endsWith(inputRef));
   }
 
@@ -1708,7 +1708,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     assertThat(r.size(), is(2));
     for (RexNode result : r) {
       final String resultString = result.toString();
-      assertThat(resultString, startsWith(EMP_QNAME));
+      assertThat(resultString, startsWith(EMP_QNAME.toString()));
       assertThat(resultString, endsWith(inputRef));
     }
 
@@ -1733,19 +1733,19 @@ public class RelMetadataTest extends SqlToRelTestBase {
     // With the union, we should get two origins
     // The first one should be the same one: join
     // The second should come from each union input
-    final Set<String> set = new HashSet<>();
+    final Set<List<String>> set = new HashSet<>();
     assertThat(r.size(), is(2));
     for (RexNode result : r) {
       assertThat(result.getKind(), is(SqlKind.PLUS));
       final RexCall call = (RexCall) result;
       assertThat(call.getOperands().size(), is(2));
       final RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
-      assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+      assertTrue(inputRef1.getQualifiedName().equals(EMP_QNAME));
       // Add join alpha to set
       set.add(inputRef1.getQualifiedName());
       assertThat(inputRef1.getIndex(), is(0));
       final RexTableInputRef inputRef2 = (RexTableInputRef) call.getOperands().get(1);
-      assertThat(inputRef2.getQualifiedName(), is(EMP_QNAME));
+      assertTrue(inputRef2.getQualifiedName().equals(EMP_QNAME));
       assertThat(inputRef2.getIndex(), is(5));
       assertThat(inputRef1.getIdentifier(), not(inputRef2.getIdentifier()));
     }
@@ -1800,7 +1800,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     RexCall call = (RexCall) predicates.pulledUpPredicates.get(0);
     assertThat(call.getOperands().size(), is(2));
     RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
-    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(inputRef1.getQualifiedName().equals(EMP_QNAME));
     assertThat(inputRef1.getIndex(), is(0));
 
     final LogicalTableScan deptScan =
@@ -1819,16 +1819,16 @@ public class RelMetadataTest extends SqlToRelTestBase {
     call = (RexCall) predicates.pulledUpPredicates.get(0);
     assertThat(call.getOperands().size(), is(2));
     inputRef1 = (RexTableInputRef) call.getOperands().get(0);
-    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(inputRef1.getQualifiedName().equals(EMP_QNAME));
     assertThat(inputRef1.getIndex(), is(0));
     // From Join
     call = (RexCall) predicates.pulledUpPredicates.get(1);
     assertThat(call.getOperands().size(), is(2));
     inputRef1 = (RexTableInputRef) call.getOperands().get(0);
-    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(inputRef1.getQualifiedName().equals(EMP_QNAME));
     assertThat(inputRef1.getIndex(), is(7));
     RexTableInputRef inputRef2 = (RexTableInputRef) call.getOperands().get(1);
-    assertThat(inputRef2.getQualifiedName(), is("[CATALOG, SALES, DEPT]"));
+    assertTrue(inputRef2.getQualifiedName().equals(ImmutableList.of("CATALOG", "SALES", "DEPT")));
     assertThat(inputRef2.getIndex(), is(0));
   }
 
@@ -1844,7 +1844,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     RexCall call = (RexCall) pulledUpPredicates.get(0);
     assertThat(call.getOperands().size(), is(2));
     final RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
-    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(inputRef1.getQualifiedName().equals(EMP_QNAME));
     assertThat(inputRef1.getIndex(), is(0));
     final RexLiteral constant = (RexLiteral) call.getOperands().get(1);
     assertThat(constant.toString(), is("5"));
@@ -1863,7 +1863,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     RexCall call = (RexCall) pulledUpPredicates.get(0);
     assertThat(call.getOperands().size(), is(2));
     final RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
-    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertTrue(inputRef1.getQualifiedName().equals(EMP_QNAME));
     assertThat(inputRef1.getIndex(), is(0));
     final RexLiteral constant = (RexLiteral) call.getOperands().get(1);
     assertThat(constant.toString(), is("5"));

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/site/_docs/materialized_views.md
----------------------------------------------------------------------
diff --git a/site/_docs/materialized_views.md b/site/_docs/materialized_views.md
index bbce64b..9856a87 100644
--- a/site/_docs/materialized_views.md
+++ b/site/_docs/materialized_views.md
@@ -33,9 +33,10 @@ For details, see the [lattices documentation]({{ site.baseurl }}/docs/lattice.ht
 
 ## Expose materialized views from adapters
 
-Some adapters have their own notion of materialized views.
+Some adapters and projects that rely on Calcite have their own notion of materialized views.
 For example, Apache Cassandra allows the user to define materialized views based on existing tables which are automatically maintained.
 The Cassandra adapter automatically exposes these materialized views to Calcite.
+Another example is Apache Hive, whose integration with Calcite materialized views is ongoing.
 By understanding some tables as materialized views, Calcite has the opportunity to automatically rewrite queries to use these views.
 
 ## View-based query rewriting
@@ -53,18 +54,34 @@ The following example is taken from the documentation of {SubstitutionVisitor}:
  * Result: `SELECT a, c FROM mv WHERE b = 4`
 
 Note that {result} uses the materialized view table {mv} and a simplified condition {b = 4}.
-This can accomplish a large number of rewritings, but only those based on star schemas.
-This type of rewriting cannot be used for more complex views.
-{MaterializedViewJoinRule} attempts to match queries to views defined using arbitrary queries.
-The logic of the rule is based on [this paper](http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.95.113).
+This can accomplish a large number of rewritings.
+However, this approach is not scalable in the presence of complex views, e.g., views containing many join operators, 
+since it relies on the planner rules to create the equivalence between expressions.
 
-There are several limitations to the current implementation:
+In turn, two alternative rules that attempt to match queries to views defined using arbitrary queries, 
+have been proposed. They are both based on the ideas of the [same paper](http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.95.113).
 
-* The query defining the view must use only inner joins
-* Only equality predicates are supported
-* Predicates on tables used in the view must exactly match predicates in the query
-* Rewriting is unoptimized and will attempt to match all views against each query
+__MaterializedViewJoinRule__ is the first alternative. There are several limitations to the current implementation:
+
+1. The query defining the view must use only inner joins
+2. Only equality predicates are supported
+3. Predicates on tables used in the view must exactly match predicates in the query
+4. Rewriting is unoptimized and will attempt to match all views against each query
 
 These limitations are not fundamental the approach however and will hopefully be removed in the future.
 Note that the rule is currently disabled by default.
 To make use of the rule, {MaterializedViewJoinRule.INSTANCE_PROJECT} and {MaterializedViewJoinRule.INSTANCE_TABLE_SCAN} need to be added to the planner.
+
+__AbstractMaterializedViewRule__ is the second alternative. It builds on the same ideas but it attempts to be more generic.
+In particular, some of the limitations of the previous rule, such as number `2.` and `3.`, do not exist for this rule.
+Additionally, the rule will be able to rewrite expressions rooted at an Aggregate operator, rolling aggregations up if necessary.
+
+However, this rule still presents some limitations too. In addition to `1.` and `4.` above, the rule presents following
+shortcomings that we plan to address with follow-up extensions:
+
+* It does not produce rewritings using Union operators, e.g., a given query could be partially answered from the
+{mv} (year = 2014) and from the query (not(year=2014)). This can be useful if {mv} is stored in a system such as
+Druid.
+* Currently query and {mv} must use the same tables.
+
+This rule is currently enabled by default.


[10/11] calcite git commit: [CALCITE-1763] Recognize lossless casts in join/aggregate materialized view rewriting rule

Posted by jc...@apache.org.
[CALCITE-1763] Recognize lossless casts in join/aggregate materialized view rewriting rule


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

Branch: refs/heads/master
Commit: 9a691a7db88f96f930a4a4609f8aea6df601a909
Parents: a2bd49c
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Apr 26 12:49:24 2017 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Apr 26 20:05:03 2017 +0100

----------------------------------------------------------------------
 .../rel/rules/AbstractMaterializedViewRule.java | 52 ++++++++++++--------
 .../calcite/test/MaterializationTest.java       | 27 ++++++++++
 2 files changed, 58 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/9a691a7d/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
index 071d9af..576c04c 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
@@ -422,7 +422,7 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             RelBuilder builder = call.builder();
             builder.push(materialization.tableRel);
             if (!compensationPred.isAlwaysTrue()) {
-              builder.filter(compensationPred);
+              builder.filter(simplify.simplify(compensationPred));
             }
             RelNode result = unify(rexBuilder, builder, builder.build(),
                 topProject, node, topViewProject, viewNode, tableMapping,
@@ -1270,9 +1270,9 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
       Map<RexTableInputRef, Set<RexTableInputRef>> equivalenceClassesMap,
       RelMetadataQuery mq) {
     Map<String, Integer> exprsLineage = new HashMap<>();
+    Map<String, Integer> exprsLineageLosslessCasts = new HashMap<>();
     for (int i = 0; i < viewExprs.size(); i++) {
-      final RexNode e = viewExprs.get(i);
-      final Set<RexNode> s = mq.getExpressionLineage(viewNode, e);
+      final Set<RexNode> s = mq.getExpressionLineage(viewNode, viewExprs.get(i));
       if (s == null) {
         // Next expression
         continue;
@@ -1281,18 +1281,18 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
       // a single expression
       assert s.size() == 1;
       // Rewrite expr to be expressed on query tables
-      exprsLineage.put(
-          RexUtil.swapTableColumnReferences(
-              rexBuilder,
-              s.iterator().next(),
-              tableMapping.inverse(),
-              equivalenceClassesMap).toString(),
-          i);
+      final RexNode e = RexUtil.swapTableColumnReferences(rexBuilder,
+          s.iterator().next(), tableMapping.inverse(), equivalenceClassesMap);
+      exprsLineage.put(e.toString(), i);
+      if (RexUtil.isLosslessCast(e)) {
+        exprsLineageLosslessCasts.put(((RexCall) e).getOperands().get(0).toString(), i);
+      }
     }
 
     List<RexNode> rewrittenExprs = new ArrayList<>(exprs.size());
     for (RexNode expr : exprs) {
-      RexNode rewrittenExpr = replaceWithOriginalReferences(rexBuilder, expr, exprsLineage);
+      RexNode rewrittenExpr = replaceWithOriginalReferences(
+          rexBuilder, viewExprs, expr, exprsLineage, exprsLineageLosslessCasts);
       if (RexUtil.containsTableInputRef(rewrittenExpr) != null) {
         // Some expressions were not present in view output
         return null;
@@ -1367,28 +1367,38 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
    * point to.
    */
   private static RexNode replaceWithOriginalReferences(final RexBuilder rexBuilder,
-      final RexNode expr, final Map<String, Integer> mapping) {
+      final List<RexNode> originalExprs, final RexNode expr, final Map<String, Integer> mapping,
+      final Map<String, Integer> mappingLosslessCasts) {
     // Currently we allow the following:
     // 1) compensation pred can be directly map to expression
     // 2) all references in compensation pred can be map to expressions
+    // We support bypassing lossless casts.
     RexShuttle visitor =
         new RexShuttle() {
           @Override public RexNode visitCall(RexCall call) {
-            Integer pos = mapping.get(call.toString());
-            if (pos != null) {
-              // Found it
-              return rexBuilder.makeInputRef(call.getType(), pos);
-            }
-            return super.visitCall(call);
+            RexNode rw = replace(call);
+            return rw != null ? rw : super.visitCall(call);
           }
 
           @Override public RexNode visitTableInputRef(RexTableInputRef inputRef) {
-            Integer pos = mapping.get(inputRef.toString());
+            RexNode rw = replace(inputRef);
+            return rw != null ? rw : super.visitTableInputRef(inputRef);
+          }
+
+          private RexNode replace(RexNode e) {
+            Integer pos = mapping.get(e.toString());
             if (pos != null) {
               // Found it
-              return rexBuilder.makeInputRef(inputRef.getType(), pos);
+              return rexBuilder.makeInputRef(e.getType(), pos);
             }
-            return super.visitTableInputRef(inputRef);
+            pos = mappingLosslessCasts.get(e.toString());
+            if (pos != null) {
+              // Found it
+              return rexBuilder.makeCast(
+                  e.getType(), rexBuilder.makeInputRef(
+                      originalExprs.get(pos).getType(), pos));
+            }
+            return null;
           }
         };
     return visitor.apply(expr);

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a691a7d/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 54ddaed..7e2c89c 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -1355,6 +1355,33 @@ public class MaterializationTest {
               + "  EnumerableTableScan(table=[[hr, m0]])"));
   }
 
+  @Test public void testJoinMaterialization5() {
+    checkMaterialize(
+      "select cast(\"empid\" as BIGINT) from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\")",
+      "select \"empid\" \"deptno\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"empid\" > 1",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):JavaType(int) NOT NULL], "
+              + "expr#2=[1], expr#3=[>($t1, $t2)], EXPR$0=[$t1], $condition=[$t3])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinMaterialization6() {
+    checkMaterialize(
+      "select cast(\"empid\" as BIGINT) from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\")",
+      "select \"empid\" \"deptno\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"empid\" = 1",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):JavaType(int) NOT NULL], "
+              + "expr#2=[CAST($t1):INTEGER NOT NULL], expr#3=[1], expr#4=[=($t2, $t3)], "
+              + "EXPR$0=[$t1], $condition=[$t4])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
   @Test public void testJoinMaterializationUKFK1() {
     checkMaterialize(
       "select \"a\".\"empid\" \"deptno\" from\n"


[07/11] calcite git commit: [CALCITE-1731] Materialized view rewriting for join and aggregate operators

Posted by jc...@apache.org.
[CALCITE-1731] Materialized view rewriting for join and aggregate operators

* Support for rewriting when view contains cardinality-preserving joins that are not present in the query

Close apache/calcite#414


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

Branch: refs/heads/master
Commit: 1f81e1353605fc2d16de9b4d56821736b0e82464
Parents: 84b49f5
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Thu Apr 20 17:39:18 2017 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Apr 26 20:02:10 2017 +0100

----------------------------------------------------------------------
 .../calcite/adapter/java/ReflectiveSchema.java  |  40 +-
 .../calcite/plan/RelOptAbstractTable.java       |   6 +
 .../org/apache/calcite/plan/RelOptTable.java    |   7 +
 .../calcite/prepare/CalcitePrepareImpl.java     |   2 +
 .../apache/calcite/prepare/RelOptTableImpl.java |   8 +
 .../calcite/rel/RelReferentialConstraint.java   |  46 ++
 .../rel/RelReferentialConstraintImpl.java       |  68 +++
 .../rel/metadata/RelMdExpressionLineage.java    |   6 +-
 .../rel/metadata/RelMdTableReferences.java      |   6 +-
 .../rel/rules/AbstractMaterializedViewRule.java | 500 ++++++++++++++-----
 .../apache/calcite/rex/RexTableInputRef.java    |  23 +-
 .../org/apache/calcite/schema/Statistic.java    |   5 +
 .../org/apache/calcite/schema/Statistics.java   |  33 +-
 .../java/org/apache/calcite/test/JdbcTest.java  |   2 +-
 .../calcite/test/MaterializationTest.java       | 322 ++++++++++--
 .../apache/calcite/test/MockCatalogReader.java  |  11 +
 .../apache/calcite/test/SqlToRelTestBase.java   |   9 +
 site/_docs/materialized_views.md                |   1 -
 18 files changed, 885 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java b/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
index 91fe3e7..0a07f35 100644
--- a/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
@@ -27,6 +27,7 @@ import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.Function;
@@ -44,9 +45,12 @@ import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.schema.impl.AbstractTableQueryable;
 import org.apache.calcite.schema.impl.ReflectiveFunctionBase;
 import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
 
 import java.lang.reflect.Constructor;
@@ -54,6 +58,7 @@ import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Type;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -90,6 +95,7 @@ public class ReflectiveSchema
     return target;
   }
 
+  @SuppressWarnings({ "rawtypes", "unchecked" })
   @Override protected Map<String, Table> getTableMap() {
     final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
     for (Field field : clazz.getFields()) {
@@ -100,7 +106,28 @@ public class ReflectiveSchema
       }
       builder.put(fieldName, table);
     }
-    return builder.build();
+    Map<String, Table> tableMap = builder.build();
+    // Unique-Key - Foreign-Key
+    for (Field field : clazz.getFields()) {
+      if (RelReferentialConstraint.class.isAssignableFrom(field.getType())) {
+        RelReferentialConstraint rc;
+        try {
+          rc = (RelReferentialConstraint) field.get(target);
+        } catch (IllegalAccessException e) {
+          throw new RuntimeException(
+              "Error while accessing field " + field, e);
+        }
+        FieldTable table =
+            (FieldTable) tableMap.get(Util.last(rc.getSourceQualifiedName()));
+        assert table != null;
+        table.statistic = Statistics.of(
+            ImmutableList.copyOf(
+                Iterables.concat(
+                    table.getStatistic().getReferentialConstraints(),
+                    Collections.singleton(rc))));
+      }
+    }
+    return tableMap;
   }
 
   @Override protected Multimap<String, Function> getFunctionMultimap() {
@@ -319,16 +346,27 @@ public class ReflectiveSchema
   /** Table based on a Java field. */
   private static class FieldTable<T> extends ReflectiveTable {
     private final Field field;
+    private Statistic statistic;
 
     FieldTable(Field field, Type elementType, Enumerable<T> enumerable) {
+      this(field, elementType, enumerable, Statistics.UNKNOWN);
+    }
+
+    FieldTable(Field field, Type elementType, Enumerable<T> enumerable,
+        Statistic statistic) {
       super(elementType, enumerable);
       this.field = field;
+      this.statistic = statistic;
     }
 
     public String toString() {
       return "Relation {field=" + field.getName() + "}";
     }
 
+    @Override public Statistic getStatistic() {
+      return statistic;
+    }
+
     @Override public Expression getExpression(SchemaPlus schema,
         String tableName, Class clazz) {
       return Expressions.field(

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
index 187a542..d640aa8 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptAbstractTable.java
@@ -21,6 +21,7 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -94,6 +95,11 @@ public abstract class RelOptAbstractTable implements RelOptTable {
     return false;
   }
 
+  // Override to define foreign keys
+  public List<RelReferentialConstraint> getReferentialConstraints() {
+    return Collections.emptyList();
+  }
+
   public RelNode toRel(ToRelContext context) {
     return LogicalTableScan.create(context.getCluster(), this);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
index 9529a5a..668dc1c 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
@@ -20,6 +20,7 @@ import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
@@ -98,6 +99,12 @@ public interface RelOptTable extends Wrapper {
   boolean isKey(ImmutableBitSet columns);
 
   /**
+   * Returns the referential constraints existing for this table. These constraints
+   * are represented over other tables using {@link RelReferentialConstraint} nodes.
+   */
+  List<RelReferentialConstraint> getReferentialConstraints();
+
+  /**
    * Generates code for this table.
    *
    * @param clazz The desired collection class; for example {@code Queryable}.

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/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 e0ae012..a5059ae 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -534,6 +534,8 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     }
     if (prepareContext.config().materializationsEnabled()) {
       planner.addRule(MaterializedViewFilterScanRule.INSTANCE);
+      planner.addRule(AbstractMaterializedViewRule.INSTANCE_PROJECT_FILTER);
+      planner.addRule(AbstractMaterializedViewRule.INSTANCE_FILTER);
       planner.addRule(AbstractMaterializedViewRule.INSTANCE_PROJECT_JOIN);
       planner.addRule(AbstractMaterializedViewRule.INSTANCE_JOIN);
       planner.addRule(AbstractMaterializedViewRule.INSTANCE_PROJECT_AGGREGATE);

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
index ccc25d8..2cc8492 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -28,6 +28,7 @@ import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributionTraitDef;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelRecordType;
@@ -284,6 +285,13 @@ public class RelOptTableImpl extends Prepare.AbstractPreparingTable {
     return false;
   }
 
+  public List<RelReferentialConstraint> getReferentialConstraints() {
+    if (table != null) {
+      return table.getStatistic().getReferentialConstraints();
+    }
+    return ImmutableList.of();
+  }
+
   public RelDataType getRowType() {
     return rowType;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraint.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraint.java b/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraint.java
new file mode 100644
index 0000000..00b8f8f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraint.java
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import org.apache.calcite.util.mapping.IntPair;
+
+import java.util.List;
+
+/**
+ * Interface for a referential constraint, i.e., Foreign-Key - Unique-Key relationship,
+ * between two tables.
+ */
+public interface RelReferentialConstraint {
+  //~ Methods ----------------------------------------------------------------
+
+  /**
+   * Returns the number of columns in the keys.
+   */
+  int getNumColumns();
+
+  /**The qualified name of the referencing table, e.g. DEPT. */
+  List<String> getSourceQualifiedName();
+
+  /** The qualified name of the referenced table, e.g. EMP. */
+  List<String> getTargetQualifiedName();
+
+  /** The (source, target) column ordinals. */
+  List<IntPair> getColumnPairs();
+
+}
+
+// End RelReferentialConstraint.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraintImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraintImpl.java b/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraintImpl.java
new file mode 100644
index 0000000..91dbd32
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/RelReferentialConstraintImpl.java
@@ -0,0 +1,68 @@
+/*
+ * 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;
+
+import org.apache.calcite.util.mapping.IntPair;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/** RelOptReferentialConstraint base implementation. */
+public class RelReferentialConstraintImpl implements RelReferentialConstraint {
+
+  private final List<String> sourceQualifiedName;
+  private final List<String> targetQualifiedName;
+  private final List<IntPair> columnPairs;
+
+  private RelReferentialConstraintImpl(List<String> sourceQualifiedName,
+      List<String> targetQualifiedName, List<IntPair> columnPairs) {
+    this.sourceQualifiedName = ImmutableList.copyOf(sourceQualifiedName);
+    this.targetQualifiedName = ImmutableList.copyOf(targetQualifiedName);
+    this.columnPairs = ImmutableList.copyOf(columnPairs);
+  }
+
+  @Override public List<String> getSourceQualifiedName() {
+    return sourceQualifiedName;
+  }
+
+  @Override public List<String> getTargetQualifiedName() {
+    return targetQualifiedName;
+  }
+
+  @Override public List<IntPair> getColumnPairs() {
+    return columnPairs;
+  }
+
+  @Override public int getNumColumns() {
+    return columnPairs.size();
+  }
+
+  public static RelReferentialConstraintImpl of(List<String> sourceQualifiedName,
+      List<String> targetQualifiedName, List<IntPair> columnPairs) {
+    return new RelReferentialConstraintImpl(
+        sourceQualifiedName, targetQualifiedName, columnPairs);
+  }
+
+  @Override public String toString() {
+    return "{ " + sourceQualifiedName + ", " + targetQualifiedName + ", "
+        + columnPairs + " }";
+  }
+
+}
+
+// End RelReferentialConstraintImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
index 6ac5cfb..87c752d 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
@@ -123,7 +123,7 @@ public class RelMdExpressionLineage
     final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
     for (int idx : inputFieldsUsed) {
       final RexNode inputRef = RexTableInputRef.of(
-          RelTableRef.of(rel.getTable().getQualifiedName(), 0),
+          RelTableRef.of(rel.getTable(), 0),
           RexInputRef.of(idx, rel.getRowType().getFieldList()));
       final Set<RexNode> originalExprs = Sets.newHashSet(inputRef);
       final RexInputRef ref = RexInputRef.of(idx, rel.getRowType().getFieldList());
@@ -233,7 +233,7 @@ public class RelMdExpressionLineage
             shift = lRefs.size();
           }
           currentTablesMapping.put(rightRef,
-              RelTableRef.of(rightRef.getQualifiedName(), shift + rightRef.getEntityNumber()));
+              RelTableRef.of(rightRef.getTable(), shift + rightRef.getEntityNumber()));
         }
         final Set<RexNode> updatedExprs = Sets.newHashSet(
             Iterables.transform(
@@ -288,7 +288,7 @@ public class RelMdExpressionLineage
             shift = lRefs.size();
           }
           currentTablesMapping.put(tableRef,
-              RelTableRef.of(tableRef.getQualifiedName(), shift + tableRef.getEntityNumber()));
+              RelTableRef.of(tableRef.getTable(), shift + tableRef.getEntityNumber()));
         }
         final Set<RexNode> updatedExprs = Sets.newHashSet(
             Iterables.transform(

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java
index 358c872..b12b425 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java
@@ -88,7 +88,7 @@ public class RelMdTableReferences
    * TableScan table reference.
    */
   public Set<RelTableRef> getTableReferences(TableScan rel, RelMetadataQuery mq) {
-    return Sets.newHashSet(RelTableRef.of(rel.getTable().getQualifiedName(), 0));
+    return Sets.newHashSet(RelTableRef.of(rel.getTable(), 0));
   }
 
   /**
@@ -123,7 +123,7 @@ public class RelMdTableReferences
         shift = lRefs.size();
       }
       RelTableRef shiftTableRef = RelTableRef.of(
-          rightRef.getQualifiedName(), shift + rightRef.getEntityNumber());
+          rightRef.getTable(), shift + rightRef.getEntityNumber());
       assert !result.contains(shiftTableRef);
       result.add(shiftTableRef);
     }
@@ -152,7 +152,7 @@ public class RelMdTableReferences
           shift = lRefs.size();
         }
         RelTableRef shiftTableRef = RelTableRef.of(
-            tableRef.getQualifiedName(), shift + tableRef.getEntityNumber());
+            tableRef.getTable(), shift + tableRef.getEntityNumber());
         assert !result.contains(shiftTableRef);
         result.add(shiftTableRef);
         currentTablesMapping.put(tableRef, shiftTableRef);

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
index fdc3774..071d9af 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
@@ -23,20 +23,20 @@ import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.SubstitutionVisitor;
 import org.apache.calcite.plan.volcano.VolcanoPlanner;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 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.core.TableScan;
 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;
 import org.apache.calcite.rex.RexCall;
@@ -54,25 +54,31 @@ import org.apache.calcite.tools.RelBuilder.AggCall;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
+import org.apache.calcite.util.graph.DefaultDirectedGraph;
+import org.apache.calcite.util.graph.DefaultEdge;
+import org.apache.calcite.util.graph.DirectedGraph;
 import org.apache.calcite.util.mapping.IntPair;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.MappingType;
 import org.apache.calcite.util.mapping.Mappings;
 import org.apache.calcite.util.trace.CalciteLogger;
 
+import org.apache.commons.lang3.tuple.ImmutableTriple;
+import org.apache.commons.lang3.tuple.Triple;
+
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableMultiset;
 import com.google.common.collect.Multimap;
-import com.google.common.collect.Multiset;
+import com.google.common.collect.Sets;
 
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
@@ -92,6 +98,12 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
   private static final CalciteLogger LOGGER =
       new CalciteLogger(LoggerFactory.getLogger(AbstractMaterializedViewRule.class));
 
+  public static final MaterializedViewProjectFilterRule INSTANCE_PROJECT_FILTER =
+      new MaterializedViewProjectFilterRule(RelFactories.LOGICAL_BUILDER);
+
+  public static final MaterializedViewOnlyFilterRule INSTANCE_FILTER =
+      new MaterializedViewOnlyFilterRule(RelFactories.LOGICAL_BUILDER);
+
   public static final MaterializedViewProjectJoinRule INSTANCE_PROJECT_JOIN =
       new MaterializedViewProjectJoinRule(RelFactories.LOGICAL_BUILDER);
 
@@ -116,33 +128,36 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
    * Rewriting logic is based on "Optimizing Queries Using Materialized Views:
    * A Practical, Scalable Solution" by Goldstein and Larson.
    *
-   * On the query side, rules matches a Project-node chain or node, where node
+   * <p>On the query side, rules matches a Project-node chain or node, where node
    * is either an Aggregate or a Join. Subplan rooted at the node operator must
    * be composed of one or more of the following operators: TableScan, Project,
    * Filter, and Join.
    *
-   * For each join MV, we need to check the following:
-   * 1) The plan rooted at the Join operator in the view produces all rows
-   * needed by the plan rooted at the Join operator in the query.
-   * 2) All columns required by compensating predicates, i.e., predicates that
-   * need to be enforced over the view, are available at the view output.
-   * 3) All output expressions can be computed from the output of the view.
-   * 4) All output rows occur with the correct duplication factor.
-   * TODO: Currently we only allow the same tables in the view and the query,
-   * thus we are sure condition 4 is met. This restriction will be lifted in
-   * the future.
+   * <p>For each join MV, we need to check the following:
+   * <ol>
+   * <li> The plan rooted at the Join operator in the view produces all rows
+   * needed by the plan rooted at the Join operator in the query.</li>
+   * <li> All columns required by compensating predicates, i.e., predicates that
+   * need to be enforced over the view, are available at the view output.</li>
+   * <li> All output expressions can be computed from the output of the view.</li>
+   * <li> All output rows occur with the correct duplication factor. We might
+   * rely on existing Unique-Key - Foreign-Key relationships to extract that
+   * information.</li>
+   * </ol>
    *
-   * In turn, for each aggregate MV, we need to check the following:
-   * 1) The plan rooted at the Aggregate operator in the view produces all rows
-   * needed by the plan rooted at the Aggregate operator in the query.
-   * 2) All columns required by compensating predicates, i.e., predicates that
-   * need to be enforced over the view, are available at the view output.
-   * 3) The grouping columns in the query are a subset of the grouping columns
-   * in the view.
-   * 4) All columns required to perform further grouping are available in the
-   * view output.
-   * 5) All columns required to compute output expressions are available in the
-   * view output.
+   * <p>In turn, for each aggregate MV, we need to check the following:
+   * <ol>
+   * <li> The plan rooted at the Aggregate operator in the view produces all rows
+   * needed by the plan rooted at the Aggregate operator in the query.</li>
+   * <li> All columns required by compensating predicates, i.e., predicates that
+   * need to be enforced over the view, are available at the view output.</li>
+   * <li> The grouping columns in the query are a subset of the grouping columns
+   * in the view.</li>
+   * <li> All columns required to perform further grouping are available in the
+   * view output.</li>
+   * <li> All columns required to compute output expressions are available in the
+   * view output.</li>
+   * </ol>
    */
   protected void perform(RelOptRuleCall call, Project topProject, RelNode node) {
     final RexBuilder rexBuilder = node.getCluster().getRexBuilder();
@@ -174,9 +189,6 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
       if (!applicableMaterializations.isEmpty()) {
         // 2. Initialize all query related auxiliary data structures
         // that will be used throughout query rewriting process
-        final Multiset<RelOptTable> qTableBag = ImmutableMultiset.copyOf(
-            RelOptUtil.findAllTables(node));
-
         // Generate query table references
         final Set<RelTableRef> queryTableRefs = mq.getTableReferences(node);
         if (queryTableRefs == null) {
@@ -191,13 +203,16 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
           // Bail out
           return;
         }
-        final RexNode[] queryPreds = splitPredicates(
-            rexBuilder, queryPredicateList.pulledUpPredicates);
+        final RexNode pred = simplify.simplify(
+            RexUtil.composeConjunction(
+                rexBuilder, queryPredicateList.pulledUpPredicates, false));
+        final Triple<RexNode, RexNode, RexNode> queryPreds =
+            splitPredicates(rexBuilder, pred);
 
         // Extract query equivalence classes. An equivalence class is a set
         // of columns in the query output that are known to be equal.
         final EquivalenceClasses qEC = new EquivalenceClasses();
-        for (RexNode conj : RelOptUtil.conjunctions(queryPreds[0])) {
+        for (RexNode conj : RelOptUtil.conjunctions(queryPreds.getLeft())) {
           assert conj.isA(SqlKind.EQUALS);
           RexCall equiCond = (RexCall) conj;
           qEC.addEquivalenceClass(
@@ -210,11 +225,6 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
         for (RelOptMaterialization materialization : applicableMaterializations) {
           final Project topViewProject;
           final RelNode viewNode;
-          // 3.1. Check whether it is a valid view
-          if (!isViewMatching(materialization.queryRel)) {
-            // Skip it
-            continue;
-          }
           if (materialization.queryRel instanceof Project) {
             topViewProject = (Project) materialization.queryRel;
             viewNode = topViewProject.getInput();
@@ -223,24 +233,14 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             viewNode = materialization.queryRel;
           }
 
-          // 3.2. View checks before proceeding
+          // 3.1. View checks before proceeding
           if (!isValidPlan(topViewProject, viewNode, mq)) {
             // Skip it
             continue;
           }
 
-          // 3.3. Initialize all query related auxiliary data structures
+          // 3.2. Initialize all query related auxiliary data structures
           // that will be used throughout query rewriting process
-          // Extract view tables
-          Multiset<RelOptTable> vTableBag = ImmutableMultiset.copyOf(
-              RelOptUtil.findAllTables(viewNode));
-          if (!qTableBag.equals(vTableBag)) {
-            // Currently we only support rewriting with views that use
-            // the same set of tables than the query, thus we skip it
-            // TODO: Extend to lift this restriction
-            continue;
-          }
-
           // Extract view predicates
           final RelOptPredicateList viewPredicateList =
               mq.getAllPredicates(viewNode);
@@ -248,8 +248,11 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             // Skip it
             continue;
           }
-          final RexNode[] viewPreds = splitPredicates(
-              rexBuilder, viewPredicateList.pulledUpPredicates);
+          final RexNode viewPred = simplify.simplify(
+              RexUtil.composeConjunction(
+                  rexBuilder, viewPredicateList.pulledUpPredicates, false));
+          final Triple<RexNode, RexNode, RexNode> viewPreds =
+              splitPredicates(rexBuilder, viewPred);
 
           // Extract view table references
           final Set<RelTableRef> viewTableRefs = mq.getTableReferences(viewNode);
@@ -258,6 +261,43 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             return;
           }
 
+          // Extract view tables
+          MatchModality matchModality;
+          Multimap<RexTableInputRef, RexTableInputRef> compensationEquiColumns =
+              ArrayListMultimap.create();
+          if (!queryTableRefs.equals(viewTableRefs)) {
+            // We try to compensate, e.g., for join queries it might be
+            // possible to join missing tables with view to compute result.
+            // Two supported cases: query tables are subset of view tables (we need to
+            // check whether they are cardinality-preserving joins), or view tables are
+            // subset of query tables (add additional tables through joins if possible)
+            if (viewTableRefs.containsAll(queryTableRefs)) {
+              matchModality = MatchModality.QUERY_PARTIAL;
+              final EquivalenceClasses vEC = new EquivalenceClasses();
+              for (RexNode conj : RelOptUtil.conjunctions(viewPreds.getLeft())) {
+                assert conj.isA(SqlKind.EQUALS);
+                RexCall equiCond = (RexCall) conj;
+                vEC.addEquivalenceClass(
+                    (RexTableInputRef) equiCond.getOperands().get(0),
+                    (RexTableInputRef) equiCond.getOperands().get(1));
+              }
+              if (!compensateQueryPartial(compensationEquiColumns,
+                  viewTableRefs, vEC, queryTableRefs)) {
+                // Cannot rewrite, skip it
+                continue;
+              }
+            } else if (queryTableRefs.containsAll(viewTableRefs)) {
+              // TODO: implement latest case
+              matchModality = MatchModality.VIEW_PARTIAL;
+              continue;
+            } else {
+              // Skip it
+              continue;
+            }
+          } else {
+            matchModality = MatchModality.COMPLETE;
+          }
+
           // 4. We map every table in the query to a view table with the same qualified
           // name.
           final Multimap<RelTableRef, RelTableRef> multiMapTables = ArrayListMultimap.create();
@@ -279,6 +319,23 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
           final List<BiMap<RelTableRef, RelTableRef>> flatListMappings =
               generateTableMappings(multiMapTables);
           for (BiMap<RelTableRef, RelTableRef> tableMapping : flatListMappings) {
+            // 4.0. If compensation equivalence classes exist, we need to add
+            // the mapping to the query mapping
+            final EquivalenceClasses currQEC = EquivalenceClasses.copy(qEC);
+            if (matchModality == MatchModality.QUERY_PARTIAL) {
+              for (Entry<RexTableInputRef, RexTableInputRef> e
+                  : compensationEquiColumns.entries()) {
+                // Copy origin
+                RelTableRef queryTableRef = tableMapping.inverse().get(e.getKey().getTableRef());
+                RexTableInputRef queryColumnRef = RexTableInputRef.of(queryTableRef,
+                    e.getKey().getIndex(), e.getKey().getType());
+                // Add to query equivalence classes and table mapping
+                currQEC.addEquivalenceClass(queryColumnRef, e.getValue());
+                tableMapping.put(
+                    e.getValue().getTableRef(), e.getValue().getTableRef()); //identity
+              }
+            }
+
             final RexNode compensationColumnsEquiPred;
             final RexNode compensationRangePred;
             final RexNode compensationResidualPred;
@@ -291,17 +348,17 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             // view predicates and check that every view equivalence class is a subset of a
             // query equivalence class: if it is not, we bail out.
             final RexNode viewColumnsEquiPred = RexUtil.swapTableReferences(
-                rexBuilder, viewPreds[0], tableMapping.inverse());
-            final EquivalenceClasses vEC = new EquivalenceClasses();
+                rexBuilder, viewPreds.getLeft(), tableMapping.inverse());
+            final EquivalenceClasses queryBasedVEC = new EquivalenceClasses();
             for (RexNode conj : RelOptUtil.conjunctions(viewColumnsEquiPred)) {
               assert conj.isA(SqlKind.EQUALS);
               RexCall equiCond = (RexCall) conj;
-              vEC.addEquivalenceClass(
+              queryBasedVEC.addEquivalenceClass(
                   (RexTableInputRef) equiCond.getOperands().get(0),
                   (RexTableInputRef) equiCond.getOperands().get(1));
             }
             compensationColumnsEquiPred = generateEquivalenceClasses(
-                rexBuilder, queryPreds[0], qEC, viewColumnsEquiPred, vEC);
+                rexBuilder, currQEC, queryBasedVEC);
             if (compensationColumnsEquiPred == null) {
               // Skip it
               continue;
@@ -310,9 +367,10 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             // 4.2. We check that range intervals for the query are contained in the view.
             // Compute compensating predicates.
             final RexNode queryRangePred = RexUtil.swapColumnReferences(
-                rexBuilder, queryPreds[1], qEC.getEquivalenceClassesMap());
+                rexBuilder, queryPreds.getMiddle(), currQEC.getEquivalenceClassesMap());
             final RexNode viewRangePred = RexUtil.swapTableColumnReferences(
-                rexBuilder, viewPreds[1], tableMapping.inverse(), qEC.getEquivalenceClassesMap());
+                rexBuilder, viewPreds.getMiddle(), tableMapping.inverse(),
+                currQEC.getEquivalenceClassesMap());
             compensationRangePred = SubstitutionVisitor.splitFilter(
                 simplify, queryRangePred, viewRangePred);
             if (compensationRangePred == null) {
@@ -324,9 +382,10 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             // within the view.
             // Compute compensating predicates.
             final RexNode queryResidualPred = RexUtil.swapColumnReferences(
-                rexBuilder, queryPreds[2], qEC.getEquivalenceClassesMap());
+                rexBuilder, queryPreds.getRight(), currQEC.getEquivalenceClassesMap());
             final RexNode viewResidualPred = RexUtil.swapTableColumnReferences(
-                rexBuilder, viewPreds[2], tableMapping.inverse(), qEC.getEquivalenceClassesMap());
+                rexBuilder, viewPreds.getRight(), tableMapping.inverse(),
+                currQEC.getEquivalenceClassesMap());
             compensationResidualPred = SubstitutionVisitor.splitFilter(
                 simplify, queryResidualPred, viewResidualPred);
             if (compensationResidualPred == null) {
@@ -348,7 +407,7 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
               List<RexNode> viewExprs = extractExpressions(topViewProject, viewNode,
                   rexBuilder);
               compensationPred = rewriteExpression(rexBuilder, viewNode, viewExprs,
-                  compensationPred, tableMapping, qEC.getEquivalenceClassesMap(), mq);
+                  compensationPred, tableMapping, currQEC.getEquivalenceClassesMap(), mq);
               if (compensationPred == null) {
                 // Skip it
                 continue;
@@ -367,7 +426,7 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             }
             RelNode result = unify(rexBuilder, builder, builder.build(),
                 topProject, node, topViewProject, viewNode, tableMapping,
-                qEC.getEquivalenceClassesMap(), mq);
+                currQEC.getEquivalenceClassesMap(), mq);
             if (result == null) {
               // Skip it
               continue;
@@ -382,16 +441,16 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
   protected abstract boolean isValidPlan(Project topProject, RelNode node,
       RelMetadataQuery mq);
 
-  protected abstract boolean isViewMatching(RelNode node);
-
   protected abstract List<RexNode> extractExpressions(Project topProject,
       RelNode node, RexBuilder rexBuilder);
 
-  /* This method is responsible for rewriting the query using the given view query.
+  /**
+   * This method is responsible for rewriting the query using the given view query.
    *
-   * The input node is a Scan on the view table and possibly a compensation Filter
+   * <p>The input node is a Scan on the view table and possibly a compensation Filter
    * on top. If a rewriting can be produced, we return that rewriting. If it cannot
-   * be produced, we will return null. */
+   * be produced, we will return null.
+   */
   protected abstract RelNode unify(RexBuilder rexBuilder, RelBuilder relBuilder,
       RelNode input, Project topProject, RelNode node,
       Project topViewProject, RelNode viewNode,
@@ -412,35 +471,19 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
 
     @Override protected boolean isValidPlan(Project topProject, RelNode node,
         RelMetadataQuery mq) {
-      Join join = (Join) node;
-      if (join.getJoinType() != JoinRelType.INNER) {
-        // TODO: Rewriting for non-inner joins not supported yet
-        return false;
-      }
-      return isValidRexNodePlan(join, mq);
-    }
-
-    @Override protected boolean isViewMatching(RelNode node) {
-      if (node instanceof Join) {
-        return true;
-      }
-      if (node instanceof Project && ((Project) node).getInput() instanceof Join) {
-        return true;
-      }
-      return false;
+      return isValidRexNodePlan(node, mq);
     }
 
     @Override protected List<RexNode> extractExpressions(Project topProject,
         RelNode node, RexBuilder rexBuilder) {
-      Join viewJoin = (Join) node;
       List<RexNode> viewExprs = new ArrayList<>();
       if (topProject != null) {
         for (RexNode e : topProject.getChildExps()) {
           viewExprs.add(e);
         }
       } else {
-        for (int i = 0; i < viewJoin.getRowType().getFieldCount(); i++) {
-          viewExprs.add(rexBuilder.makeInputRef(viewJoin, i));
+        for (int i = 0; i < node.getRowType().getFieldCount(); i++) {
+          viewExprs.add(rexBuilder.makeInputRef(node, i));
         }
       }
       return viewExprs;
@@ -498,6 +541,23 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     }
   }
 
+  /** Rule that matches Project on Filter. */
+  public static class MaterializedViewProjectFilterRule extends MaterializedViewJoinRule {
+    public MaterializedViewProjectFilterRule(RelBuilderFactory relBuilderFactory) {
+      super(
+          operand(Project.class,
+              operand(Filter.class, any())),
+          relBuilderFactory,
+          "MaterializedViewJoinRule(Project-Filter)");
+    }
+
+    @Override public void onMatch(RelOptRuleCall call) {
+      final Project project = call.rel(0);
+      final Filter filter = call.rel(1);
+      perform(call, project, filter);
+    }
+  }
+
   /** Rule that matches Join. */
   public static class MaterializedViewOnlyJoinRule extends MaterializedViewJoinRule {
     public MaterializedViewOnlyJoinRule(RelBuilderFactory relBuilderFactory) {
@@ -513,6 +573,21 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     }
   }
 
+  /** Rule that matches Filter. */
+  public static class MaterializedViewOnlyFilterRule extends MaterializedViewJoinRule {
+    public MaterializedViewOnlyFilterRule(RelBuilderFactory relBuilderFactory) {
+      super(
+          operand(Filter.class, any()),
+          relBuilderFactory,
+          "MaterializedViewJoinRule(Filter)");
+    }
+
+    @Override public void onMatch(RelOptRuleCall call) {
+      final Filter filter = call.rel(0);
+      perform(call, null, filter);
+    }
+  }
+
   //~ Instances Aggregate ----------------------------------------------------
 
   /** Materialized view rewriting for aggregate */
@@ -526,6 +601,9 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
 
     @Override protected boolean isValidPlan(Project topProject, RelNode node,
         RelMetadataQuery mq) {
+      if (!(node instanceof Aggregate)) {
+        return false;
+      }
       Aggregate aggregate = (Aggregate) node;
       if (aggregate.getGroupType() != Aggregate.Group.SIMPLE) {
         // TODO: Rewriting with grouping sets not supported yet
@@ -534,16 +612,6 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
       return isValidRexNodePlan(aggregate.getInput(), mq);
     }
 
-    @Override protected boolean isViewMatching(RelNode node) {
-      if (node instanceof Aggregate) {
-        return true;
-      }
-      if (node instanceof Project && ((Project) node).getInput() instanceof Aggregate) {
-        return true;
-      }
-      return false;
-    }
-
     @Override protected List<RexNode> extractExpressions(Project topProject,
         RelNode node, RexBuilder rexBuilder) {
       Aggregate viewAggregate = (Aggregate) node;
@@ -638,7 +706,10 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
           continue;
         }
         AggregateCall queryAggCall = queryAggregate.getAggCallList().get(idx);
-        List<Integer> queryAggCallIndexes = queryAggCall.getArgList();
+        List<Integer> queryAggCallIndexes = new ArrayList<>();
+        for (int aggCallIdx : queryAggCall.getArgList()) {
+          queryAggCallIndexes.add(m.get(aggCallIdx).iterator().next());
+        }
         for (int j = 0; j < viewAggregate.getAggCallList().size(); j++) {
           AggregateCall viewAggCall = viewAggregate.getAggCallList().get(j);
           if (queryAggCall.getAggregation() != viewAggCall.getAggregation()
@@ -648,11 +719,7 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
             // Continue
             continue;
           }
-          List<Integer> viewAggCallIndexes = new ArrayList<>();
-          for (int aggCallIdx : viewAggCall.getArgList()) {
-            viewAggCallIndexes.add(m.get(aggCallIdx).iterator().next());
-          }
-          if (!queryAggCallIndexes.equals(viewAggCallIndexes)) {
+          if (!queryAggCallIndexes.equals(viewAggCall.getArgList())) {
             // Continue
             continue;
           }
@@ -864,9 +931,11 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
 
   //~ Methods ----------------------------------------------------------------
 
-  /* It will flatten a multimap containing table references to table references,
+  /**
+   * It will flatten a multimap containing table references to table references,
    * producing all possible combinations of mappings. Each of the mappings will
-   * be bi-directional. */
+   * be bi-directional.
+   */
   private static List<BiMap<RelTableRef, RelTableRef>> generateTableMappings(
       Multimap<RelTableRef, RelTableRef> multiMapTables) {
     final List<BiMap<RelTableRef, RelTableRef>> result = new ArrayList<>();
@@ -897,7 +966,7 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     return result;
   }
 
-  /* Currently we only support TableScan - Project - Filter - Join */
+  /** Currently we only support TableScan - Project - Filter - Join */
   private static boolean isValidRexNodePlan(RelNode node, RelMetadataQuery mq) {
     final Multimap<Class<? extends RelNode>, RelNode> m =
             mq.getNodeTypes(node);
@@ -913,22 +982,26 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     return true;
   }
 
-  /* Classifies each of the predicates in the list into one of these three
+  /**
+   * Classifies each of the predicates in the list into one of these three
    * categories:
-   * - column equality predicates, or
-   * - range predicates, comprising <, <=, >, >=, and = between a reference
-   * and a constant, or
-   * - residual predicates, all the rest
+   * <ul>
+   * <li> 1-l) column equality predicates, or </li>
+   * <li> 2-m) range predicates, comprising <, <=, >, >=, and = between a reference
+   * and a constant, or </li>
+   * <li> 3-r) residual predicates, all the rest</li>
+   * </ul>
    *
-   * For each category, it creates the conjunction of the predicates. The
+   * <p>For each category, it creates the conjunction of the predicates. The
    * result is an array of three RexNode objects corresponding to each
-   * category. */
-  private static RexNode[] splitPredicates(
-      RexBuilder rexBuilder, ImmutableList<RexNode> predicates) {
+   * category.
+   */
+  private static Triple<RexNode, RexNode, RexNode> splitPredicates(
+      RexBuilder rexBuilder, RexNode pred) {
     List<RexNode> equiColumnsPreds = new ArrayList<>();
     List<RexNode> rangePreds = new ArrayList<>();
     List<RexNode> residualPreds = new ArrayList<>();
-    for (RexNode e : predicates) {
+    for (RexNode e : RelOptUtil.conjunctions(pred)) {
       switch (e.getKind()) {
       case EQUALS:
         RexCall eqCall = (RexCall) e;
@@ -963,33 +1036,144 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
         residualPreds.add(e);
       }
     }
-    return new RexNode[] {
+    return ImmutableTriple.<RexNode, RexNode, RexNode>of(
         RexUtil.composeConjunction(rexBuilder, equiColumnsPreds, false),
         RexUtil.composeConjunction(rexBuilder, rangePreds, false),
-        RexUtil.composeConjunction(rexBuilder, residualPreds, false)};
+        RexUtil.composeConjunction(rexBuilder, residualPreds, false));
+  }
+
+  /**
+   * It checks whether the query can be rewritten using the view even though the
+   * view uses additional tables. In order to do that, we need to double-check
+   * that every join that exists in the view and is not in the query is a
+   * cardinality-preserving join, i.e., it only appends columns to the row
+   * without changing its multiplicity. Thus, the join needs to be:
+   * <ul>
+   * <li> Equi-join </li>
+   * <li> Between all columns in the keys </li>
+   * <li> Foreign-key columns do not allow NULL values </li>
+   * <li> Foreign-key </li>
+   * <li> Unique-key </li>
+   * </ul>
+   *
+   * <p>If it can be rewritten, it returns true and it inserts the missing equi-join
+   * predicates in the input compensationEquiColumns multimap. Otherwise, it returns
+   * false.
+   */
+  private static boolean compensateQueryPartial(
+      Multimap<RexTableInputRef, RexTableInputRef> compensationEquiColumns,
+      Set<RelTableRef> viewTableRefs, EquivalenceClasses vEC, Set<RelTableRef> queryTableRefs) {
+    // Create UK-FK graph with view tables
+    final DirectedGraph<RelTableRef, Edge> graph =
+        DefaultDirectedGraph.create(Edge.FACTORY);
+    final Multimap<List<String>, RelTableRef> tableQNameToTableRefs =
+        ArrayListMultimap.create();
+    final Set<RelTableRef> extraTableRefs = new HashSet<>();
+    for (RelTableRef tRef : viewTableRefs) {
+      // Add tables in view as vertices
+      graph.addVertex(tRef);
+      tableQNameToTableRefs.put(tRef.getQualifiedName(), tRef);
+      if (!queryTableRefs.contains(tRef)) {
+        // Add to extra tables if table is not part of the query
+        extraTableRefs.add(tRef);
+      }
+    }
+    for (RelTableRef tRef : graph.vertexSet()) {
+      // Add edges between tables
+      List<RelReferentialConstraint> constraints =
+          tRef.getTable().getReferentialConstraints();
+      for (RelReferentialConstraint constraint : constraints) {
+        Collection<RelTableRef> parentTableRefs =
+            tableQNameToTableRefs.get(constraint.getTargetQualifiedName());
+        if (parentTableRefs == null || parentTableRefs.isEmpty()) {
+          continue;
+        }
+        for (RelTableRef parentTRef : parentTableRefs) {
+          boolean canBeRewritten = true;
+          Multimap<RexTableInputRef, RexTableInputRef> equiColumns =
+                  ArrayListMultimap.create();
+          for (int pos = 0; pos < constraint.getNumColumns(); pos++) {
+            int foreignKeyPos = constraint.getColumnPairs().get(pos).source;
+            RelDataType foreignKeyColumnType =
+                tRef.getTable().getRowType().getFieldList().get(foreignKeyPos).getType();
+            RexTableInputRef foreignKeyColumnRef =
+                RexTableInputRef.of(tRef, foreignKeyPos, foreignKeyColumnType);
+            int uniqueKeyPos = constraint.getColumnPairs().get(pos).target;
+            RexTableInputRef uniqueKeyColumnRef = RexTableInputRef.of(parentTRef, uniqueKeyPos,
+                parentTRef.getTable().getRowType().getFieldList().get(uniqueKeyPos).getType());
+            if (!foreignKeyColumnType.isNullable()
+                && vEC.getEquivalenceClassesMap().get(uniqueKeyColumnRef).contains(
+                    foreignKeyColumnRef)) {
+              equiColumns.put(foreignKeyColumnRef, uniqueKeyColumnRef);
+            } else {
+              canBeRewritten = false;
+              break;
+            }
+          }
+          if (canBeRewritten) {
+            // Add edge FK -> UK
+            Edge edge = graph.getEdge(tRef, parentTRef);
+            if (edge == null) {
+              edge = graph.addEdge(tRef, parentTRef);
+            }
+            edge.equiColumns.putAll(equiColumns);
+            break;
+          }
+        }
+      }
+    }
+
+    // Try to eliminate tables from graph: if we can do it, it means extra tables in
+    // view are cardinality-preserving joins
+    boolean done = false;
+    do {
+      List<RelTableRef> nodesToRemove = new ArrayList<>();
+      for (RelTableRef tRef : graph.vertexSet()) {
+        if (graph.getInwardEdges(tRef).size() == 1
+            && graph.getOutwardEdges(tRef).isEmpty()) {
+          // UK-FK join
+          nodesToRemove.add(tRef);
+          if (extraTableRefs.contains(tRef)) {
+            // We need to add to compensation columns as the table is not present in the query
+            compensationEquiColumns.putAll(graph.getInwardEdges(tRef).get(0).equiColumns);
+          }
+        }
+      }
+      if (!nodesToRemove.isEmpty()) {
+        graph.removeAllVertices(nodesToRemove);
+      } else {
+        done = true;
+      }
+    } while (!done);
+
+    // After removing them, we check whether all the remaining tables in the graph
+    // are tables present in the query: if they are, we can try to rewrite
+    if (!Collections.disjoint(graph.vertexSet(), extraTableRefs)) {
+      return false;
+    }
+    return true;
   }
 
-  /* Given the equi-column predicates of the query and the view and the
+  /**
+   * Given the equi-column predicates of the query and the view and the
    * computed equivalence classes, it extracts possible mappings between
    * the equivalence classes.
    *
-   * If there is no mapping, it returns null. If there is a exact match,
+   * <p>If there is no mapping, it returns null. If there is a exact match,
    * it will return a compensation predicate that evaluates to true.
    * Finally, if a compensation predicate needs to be enforced on top of
    * the view to make the equivalences classes match, it returns that
-   * compensation predicate */
+   * compensation predicate
+   */
   private static RexNode generateEquivalenceClasses(RexBuilder rexBuilder,
-      RexNode queryEquiColumnsPreds, EquivalenceClasses qEC,
-      RexNode viewEquiColumnsPreds, EquivalenceClasses vEC) {
-    if (queryEquiColumnsPreds.isAlwaysTrue() && viewEquiColumnsPreds.isAlwaysTrue()) {
+      EquivalenceClasses qEC, EquivalenceClasses vEC) {
+    if (qEC.getEquivalenceClasses().isEmpty() && vEC.getEquivalenceClasses().isEmpty()) {
       // No column equality predicates in query and view
       // Empty mapping and compensation predicate
-      assert qEC.getEquivalenceClasses().isEmpty() && vEC.getEquivalenceClasses().isEmpty();
       return rexBuilder.makeLiteral(true);
     }
-    if (queryEquiColumnsPreds.isAlwaysTrue() || viewEquiColumnsPreds.isAlwaysTrue()) {
+    if (qEC.getEquivalenceClasses().isEmpty() || vEC.getEquivalenceClasses().isEmpty()) {
       // No column equality predicates in query or view
-      assert qEC.getEquivalenceClasses().isEmpty() || vEC.getEquivalenceClasses().isEmpty();
       return null;
     }
 
@@ -1020,11 +1204,13 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     return compensationPredicate;
   }
 
-  /* Given the query and view equivalence classes, it extracts the possible mappings
+  /**
+   * Given the query and view equivalence classes, it extracts the possible mappings
    * from each view equivalence class to each query equivalence class.
    *
-   * If any of the view equivalence classes cannot be mapped to a query equivalence
-   * class, it returns null. */
+   * <p>If any of the view equivalence classes cannot be mapped to a query equivalence
+   * class, it returns null.
+   */
   private static Mapping extractPossibleMapping(
       List<Set<RexTableInputRef>> queryEquivalenceClasses,
       List<Set<RexTableInputRef>> viewEquivalenceClasses) {
@@ -1051,10 +1237,11 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     return mapping;
   }
 
-  /* Given the input expression that references source expressions in the query,
+  /**
+   * Given the input expression that references source expressions in the query,
    * it will rewrite it to refer to the view output.
    *
-   * If any of the subexpressions in the input expression cannot be mapped to
+   * <p>If any of the subexpressions in the input expression cannot be mapped to
    * the query, it will return null.
    */
   private static RexNode rewriteExpression(
@@ -1115,9 +1302,11 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     return rewrittenExprs;
   }
 
-  /* Mapping from node expressions to target expressions.
+  /**
+   * Mapping from node expressions to target expressions.
    *
-   * If any of the expressions cannot be mapped, we return null. */
+   * <p>If any of the expressions cannot be mapped, we return null.
+   */
   private static Multimap<Integer, Integer> generateMapping(
       RexBuilder rexBuilder,
       RelNode node,
@@ -1171,10 +1360,12 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     return m;
   }
 
-  /* Given the input expression, it will replace (sub)expressions when possible
+  /**
+   * Given the input expression, it will replace (sub)expressions when possible
    * using the content of the mapping. In particular, the mapping contains the
    * digest of the expression and the index that the replacement input ref should
-   * point to. */
+   * point to.
+   */
   private static RexNode replaceWithOriginalReferences(final RexBuilder rexBuilder,
       final RexNode expr, final Map<String, Integer> mapping) {
     // Currently we allow the following:
@@ -1203,9 +1394,11 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     return visitor.apply(expr);
   }
 
-  /* Replaces all the input references by the position in the
+  /**
+   * Replaces all the input references by the position in the
    * input column set. If a reference index cannot be found in
-   * the input set, then we return null. */
+   * the input set, then we return null.
+   */
   private static RexNode shuttleReferences(final RexBuilder rexBuilder,
       final RexNode node, final Mapping mapping) {
     try {
@@ -1277,7 +1470,42 @@ public abstract class AbstractMaterializedViewRule extends RelOptRule {
     protected List<Set<RexTableInputRef>> getEquivalenceClasses() {
       return ImmutableList.copyOf(nodeToEquivalenceClass.values());
     }
+
+    protected static EquivalenceClasses copy(EquivalenceClasses ec) {
+      final EquivalenceClasses newEc = new EquivalenceClasses();
+      for (Entry<RexTableInputRef, Set<RexTableInputRef>> e
+          : ec.nodeToEquivalenceClass.entrySet()) {
+        newEc.nodeToEquivalenceClass.put(
+            e.getKey(), Sets.newLinkedHashSet(e.getValue()));
+      }
+      return newEc;
+    }
+  }
+
+  /** Edge for graph */
+  private static class Edge extends DefaultEdge {
+    public static final DirectedGraph.EdgeFactory<RelTableRef, Edge> FACTORY =
+        new DirectedGraph.EdgeFactory<RelTableRef, Edge>() {
+          public Edge createEdge(RelTableRef source, RelTableRef target) {
+            return new Edge(source, target);
+          }
+        };
+
+    final Multimap<RexTableInputRef, RexTableInputRef> equiColumns =
+        ArrayListMultimap.create();
+
+    public Edge(RelTableRef source, RelTableRef target) {
+      super(source, target);
+    }
   }
+
+  /** Complete, view partial, or query partial. */
+  private enum MatchModality {
+    COMPLETE,
+    VIEW_PARTIAL,
+    QUERY_PARTIAL
+  }
+
 }
 
 // End AbstractMaterializedViewRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
index b986684..a8d9fe0 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
@@ -16,11 +16,10 @@
  */
 package org.apache.calcite.rex;
 
+import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 
-import com.google.common.collect.ImmutableList;
-
 import java.util.List;
 
 /**
@@ -98,14 +97,14 @@ public class RexTableInputRef extends RexInputRef {
   /** Identifies uniquely a table by its qualified name and its entity number (occurrence) */
   public static class RelTableRef {
 
-    private final List<String> qualifiedName;
+    private final RelOptTable table;
     private final int entityNumber;
     private final String digest;
 
-    private RelTableRef(List<String> qualifiedName, int entityNumber) {
-      this.qualifiedName = ImmutableList.copyOf(qualifiedName);
+    private RelTableRef(RelOptTable table, int entityNumber) {
+      this.table = table;
       this.entityNumber = entityNumber;
-      this.digest = qualifiedName + ".#" + entityNumber;
+      this.digest = table.getQualifiedName() + ".#" + entityNumber;
     }
 
     //~ Methods ----------------------------------------------------------------
@@ -113,7 +112,7 @@ public class RexTableInputRef extends RexInputRef {
     @Override public boolean equals(Object obj) {
       return this == obj
           || obj instanceof RelTableRef
-          && qualifiedName.equals(((RelTableRef) obj).qualifiedName)
+          && table.getQualifiedName().equals(((RelTableRef) obj).getQualifiedName())
           && entityNumber == ((RelTableRef) obj).entityNumber;
     }
 
@@ -121,8 +120,12 @@ public class RexTableInputRef extends RexInputRef {
       return digest.hashCode();
     }
 
+    public RelOptTable getTable() {
+      return table;
+    }
+
     public List<String> getQualifiedName() {
-      return qualifiedName;
+      return table.getQualifiedName();
     }
 
     public int getEntityNumber() {
@@ -133,8 +136,8 @@ public class RexTableInputRef extends RexInputRef {
       return digest;
     }
 
-    public static RelTableRef of(List<String> qualifiedName, int entityNumber) {
-      return new RelTableRef(qualifiedName, entityNumber);
+    public static RelTableRef of(RelOptTable table, int entityNumber) {
+      return new RelTableRef(table, entityNumber);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/schema/Statistic.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Statistic.java b/core/src/main/java/org/apache/calcite/schema/Statistic.java
index ced3bae..4b53d89 100644
--- a/core/src/main/java/org/apache/calcite/schema/Statistic.java
+++ b/core/src/main/java/org/apache/calcite/schema/Statistic.java
@@ -18,6 +18,7 @@ package org.apache.calcite.schema;
 
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.util.ImmutableBitSet;
 
 import java.util.List;
@@ -38,6 +39,10 @@ public interface Statistic {
    */
   boolean isKey(ImmutableBitSet columns);
 
+  /** Returns the collection of referential constraints (foreign-keys)
+   * for this table. */
+  List<RelReferentialConstraint> getReferentialConstraints();
+
   /** Returns the collections of columns on which this table is sorted. */
   List<RelCollation> getCollations();
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/main/java/org/apache/calcite/schema/Statistics.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Statistics.java b/core/src/main/java/org/apache/calcite/schema/Statistics.java
index fe5a091..d3d4173 100644
--- a/core/src/main/java/org/apache/calcite/schema/Statistics.java
+++ b/core/src/main/java/org/apache/calcite/schema/Statistics.java
@@ -19,6 +19,7 @@ package org.apache.calcite.schema;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributionTraitDef;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
@@ -43,6 +44,10 @@ public class Statistics {
           return false;
         }
 
+        public List<RelReferentialConstraint> getReferentialConstraints() {
+          return ImmutableList.<RelReferentialConstraint>of();
+        }
+
         public List<RelCollation> getCollations() {
           return ImmutableList.of();
         }
@@ -52,15 +57,33 @@ public class Statistics {
         }
       };
 
+  /** Returns a statistic with a given set of referential constraints. */
+  public static Statistic of(final List<RelReferentialConstraint> referentialConstraints) {
+    return of(null, ImmutableList.<ImmutableBitSet>of(),
+        referentialConstraints, ImmutableList.<RelCollation>of());
+  }
+
   /** Returns a statistic with a given row count and set of unique keys. */
   public static Statistic of(final double rowCount,
       final List<ImmutableBitSet> keys) {
-    return of(rowCount, keys, ImmutableList.<RelCollation>of());
+    return of(rowCount, keys, ImmutableList.<RelReferentialConstraint>of(),
+        ImmutableList.<RelCollation>of());
   }
 
-  /** Returns a statistic with a given row count and set of unique keys. */
+  /** Returns a statistic with a given row count, set of unique keys,
+   * and collations. */
   public static Statistic of(final double rowCount,
-      final List<ImmutableBitSet> keys, final List<RelCollation> collations) {
+      final List<ImmutableBitSet> keys,
+      final List<RelCollation> collations) {
+    return of(rowCount, keys, ImmutableList.<RelReferentialConstraint>of(), collations);
+  }
+
+  /** Returns a statistic with a given row count, set of unique keys,
+   * referential constraints, and collations. */
+  public static Statistic of(final Double rowCount,
+      final List<ImmutableBitSet> keys,
+      final List<RelReferentialConstraint> referentialConstraints,
+      final List<RelCollation> collations) {
     return new Statistic() {
       public Double getRowCount() {
         return rowCount;
@@ -75,6 +98,10 @@ public class Statistics {
         return false;
       }
 
+      public List<RelReferentialConstraint> getReferentialConstraints() {
+        return referentialConstraints;
+      }
+
       public List<RelCollation> getCollations() {
         return collations;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/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 6819052..62a526b 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -4536,7 +4536,7 @@ public class JdbcTest {
             "select \"deptno\", \"employees\"[1] as e from \"hr\".\"depts\"\n").returnsUnordered(
         "deptno=10; E={100, 10, Bill, 10000.0, 1000}",
         "deptno=30; E=null",
-            "deptno=40; E={200, 20, Eric, 8000.0, 500}");
+        "deptno=40; E={200, 20, Eric, 8000.0, 500}");
   }
 
   @Test public void testVarcharEquals() {


[02/11] calcite git commit: [CALCITE-1731] Materialized view rewriting for join and aggregate operators

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
new file mode 100644
index 0000000..fdc3774
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
@@ -0,0 +1,1283 @@
+/*
+ * 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.RelOptMaterialization;
+import org.apache.calcite.plan.RelOptMaterializations;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.SubstitutionVisitor;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+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.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.core.TableScan;
+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.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSimplify;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexTableInputRef.RelTableRef;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilder.AggCall;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.IntPair;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.MappingType;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.calcite.util.trace.CalciteLogger;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultiset;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multiset;
+
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+/**
+ * Planner rule that converts a {@link org.apache.calcite.rel.core.Project}
+ * followed by {@link org.apache.calcite.rel.core.Aggregate} or an
+ * {@link org.apache.calcite.rel.core.Aggregate} to a scan (and possibly
+ * other operations) over a materialized view.
+ */
+public abstract class AbstractMaterializedViewRule extends RelOptRule {
+
+  private static final CalciteLogger LOGGER =
+      new CalciteLogger(LoggerFactory.getLogger(AbstractMaterializedViewRule.class));
+
+  public static final MaterializedViewProjectJoinRule INSTANCE_PROJECT_JOIN =
+      new MaterializedViewProjectJoinRule(RelFactories.LOGICAL_BUILDER);
+
+  public static final MaterializedViewOnlyJoinRule INSTANCE_JOIN =
+      new MaterializedViewOnlyJoinRule(RelFactories.LOGICAL_BUILDER);
+
+  public static final MaterializedViewProjectAggregateRule INSTANCE_PROJECT_AGGREGATE =
+      new MaterializedViewProjectAggregateRule(RelFactories.LOGICAL_BUILDER);
+
+  public static final MaterializedViewOnlyAggregateRule INSTANCE_AGGREGATE =
+      new MaterializedViewOnlyAggregateRule(RelFactories.LOGICAL_BUILDER);
+
+  //~ Constructors -----------------------------------------------------------
+
+  /** Creates a AbstractMaterializedViewRule. */
+  protected AbstractMaterializedViewRule(RelOptRuleOperand operand,
+      RelBuilderFactory relBuilderFactory, String description) {
+    super(operand, relBuilderFactory, description);
+  }
+
+  /**
+   * Rewriting logic is based on "Optimizing Queries Using Materialized Views:
+   * A Practical, Scalable Solution" by Goldstein and Larson.
+   *
+   * On the query side, rules matches a Project-node chain or node, where node
+   * is either an Aggregate or a Join. Subplan rooted at the node operator must
+   * be composed of one or more of the following operators: TableScan, Project,
+   * Filter, and Join.
+   *
+   * For each join MV, we need to check the following:
+   * 1) The plan rooted at the Join operator in the view produces all rows
+   * needed by the plan rooted at the Join operator in the query.
+   * 2) All columns required by compensating predicates, i.e., predicates that
+   * need to be enforced over the view, are available at the view output.
+   * 3) All output expressions can be computed from the output of the view.
+   * 4) All output rows occur with the correct duplication factor.
+   * TODO: Currently we only allow the same tables in the view and the query,
+   * thus we are sure condition 4 is met. This restriction will be lifted in
+   * the future.
+   *
+   * In turn, for each aggregate MV, we need to check the following:
+   * 1) The plan rooted at the Aggregate operator in the view produces all rows
+   * needed by the plan rooted at the Aggregate operator in the query.
+   * 2) All columns required by compensating predicates, i.e., predicates that
+   * need to be enforced over the view, are available at the view output.
+   * 3) The grouping columns in the query are a subset of the grouping columns
+   * in the view.
+   * 4) All columns required to perform further grouping are available in the
+   * view output.
+   * 5) All columns required to compute output expressions are available in the
+   * view output.
+   */
+  protected void perform(RelOptRuleCall call, Project topProject, RelNode node) {
+    final RexBuilder rexBuilder = node.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final RelOptPlanner planner = call.getPlanner();
+    final RexSimplify simplify =
+        new RexSimplify(rexBuilder, true,
+            planner.getExecutor() != null ? planner.getExecutor() : RexUtil.EXECUTOR);
+
+    final List<RelOptMaterialization> materializations =
+        (planner instanceof VolcanoPlanner)
+            ? ((VolcanoPlanner) planner).getMaterializations()
+            : ImmutableList.<RelOptMaterialization>of();
+
+    if (!materializations.isEmpty()) {
+      // 1. Explore query plan to recognize whether preconditions to
+      // try to generate a rewriting are met
+      if (!isValidPlan(topProject, node, mq)) {
+        return;
+      }
+
+      // Obtain applicable (filtered) materializations
+      // TODO: Filtering of relevant materializations needs to be
+      // improved so we gather only materializations that might
+      // actually generate a valid rewriting.
+      final List<RelOptMaterialization> applicableMaterializations =
+          RelOptMaterializations.getApplicableMaterializations(node, materializations);
+
+      if (!applicableMaterializations.isEmpty()) {
+        // 2. Initialize all query related auxiliary data structures
+        // that will be used throughout query rewriting process
+        final Multiset<RelOptTable> qTableBag = ImmutableMultiset.copyOf(
+            RelOptUtil.findAllTables(node));
+
+        // Generate query table references
+        final Set<RelTableRef> queryTableRefs = mq.getTableReferences(node);
+        if (queryTableRefs == null) {
+          // Bail out
+          return;
+        }
+
+        // Extract query predicates
+        final RelOptPredicateList queryPredicateList =
+            mq.getAllPredicates(node);
+        if (queryPredicateList == null) {
+          // Bail out
+          return;
+        }
+        final RexNode[] queryPreds = splitPredicates(
+            rexBuilder, queryPredicateList.pulledUpPredicates);
+
+        // Extract query equivalence classes. An equivalence class is a set
+        // of columns in the query output that are known to be equal.
+        final EquivalenceClasses qEC = new EquivalenceClasses();
+        for (RexNode conj : RelOptUtil.conjunctions(queryPreds[0])) {
+          assert conj.isA(SqlKind.EQUALS);
+          RexCall equiCond = (RexCall) conj;
+          qEC.addEquivalenceClass(
+              (RexTableInputRef) equiCond.getOperands().get(0),
+              (RexTableInputRef) equiCond.getOperands().get(1));
+        }
+
+        // 3. We iterate through all applicable materializations trying to
+        // rewrite the given query
+        for (RelOptMaterialization materialization : applicableMaterializations) {
+          final Project topViewProject;
+          final RelNode viewNode;
+          // 3.1. Check whether it is a valid view
+          if (!isViewMatching(materialization.queryRel)) {
+            // Skip it
+            continue;
+          }
+          if (materialization.queryRel instanceof Project) {
+            topViewProject = (Project) materialization.queryRel;
+            viewNode = topViewProject.getInput();
+          } else {
+            topViewProject = null;
+            viewNode = materialization.queryRel;
+          }
+
+          // 3.2. View checks before proceeding
+          if (!isValidPlan(topViewProject, viewNode, mq)) {
+            // Skip it
+            continue;
+          }
+
+          // 3.3. Initialize all query related auxiliary data structures
+          // that will be used throughout query rewriting process
+          // Extract view tables
+          Multiset<RelOptTable> vTableBag = ImmutableMultiset.copyOf(
+              RelOptUtil.findAllTables(viewNode));
+          if (!qTableBag.equals(vTableBag)) {
+            // Currently we only support rewriting with views that use
+            // the same set of tables than the query, thus we skip it
+            // TODO: Extend to lift this restriction
+            continue;
+          }
+
+          // Extract view predicates
+          final RelOptPredicateList viewPredicateList =
+              mq.getAllPredicates(viewNode);
+          if (viewPredicateList == null) {
+            // Skip it
+            continue;
+          }
+          final RexNode[] viewPreds = splitPredicates(
+              rexBuilder, viewPredicateList.pulledUpPredicates);
+
+          // Extract view table references
+          final Set<RelTableRef> viewTableRefs = mq.getTableReferences(viewNode);
+          if (viewTableRefs == null) {
+            // Bail out
+            return;
+          }
+
+          // 4. We map every table in the query to a view table with the same qualified
+          // name.
+          final Multimap<RelTableRef, RelTableRef> multiMapTables = ArrayListMultimap.create();
+          for (RelTableRef queryTableRef : queryTableRefs) {
+            for (RelTableRef viewTableRef : viewTableRefs) {
+              if (queryTableRef.getQualifiedName().equals(
+                  viewTableRef.getQualifiedName())) {
+                multiMapTables.put(queryTableRef, viewTableRef);
+              }
+            }
+          }
+
+          // If a table is used multiple times, we will create multiple mappings,
+          // and we will try to rewrite the query using each of the mappings.
+          // Then, we will try to map every source table (query) to a target
+          // table (view), and if we are successful, we will try to create
+          // compensation predicates to filter the view results further
+          // (if needed).
+          final List<BiMap<RelTableRef, RelTableRef>> flatListMappings =
+              generateTableMappings(multiMapTables);
+          for (BiMap<RelTableRef, RelTableRef> tableMapping : flatListMappings) {
+            final RexNode compensationColumnsEquiPred;
+            final RexNode compensationRangePred;
+            final RexNode compensationResidualPred;
+
+            // 4.1. Establish relationship between view and query equivalence classes.
+            // If every view equivalence class is not a subset of a query
+            // equivalence class, we bail out.
+            // To establish relationship, we swap column references of the view predicates
+            // to point to query tables. Then, we create the equivalence classes for the
+            // view predicates and check that every view equivalence class is a subset of a
+            // query equivalence class: if it is not, we bail out.
+            final RexNode viewColumnsEquiPred = RexUtil.swapTableReferences(
+                rexBuilder, viewPreds[0], tableMapping.inverse());
+            final EquivalenceClasses vEC = new EquivalenceClasses();
+            for (RexNode conj : RelOptUtil.conjunctions(viewColumnsEquiPred)) {
+              assert conj.isA(SqlKind.EQUALS);
+              RexCall equiCond = (RexCall) conj;
+              vEC.addEquivalenceClass(
+                  (RexTableInputRef) equiCond.getOperands().get(0),
+                  (RexTableInputRef) equiCond.getOperands().get(1));
+            }
+            compensationColumnsEquiPred = generateEquivalenceClasses(
+                rexBuilder, queryPreds[0], qEC, viewColumnsEquiPred, vEC);
+            if (compensationColumnsEquiPred == null) {
+              // Skip it
+              continue;
+            }
+
+            // 4.2. We check that range intervals for the query are contained in the view.
+            // Compute compensating predicates.
+            final RexNode queryRangePred = RexUtil.swapColumnReferences(
+                rexBuilder, queryPreds[1], qEC.getEquivalenceClassesMap());
+            final RexNode viewRangePred = RexUtil.swapTableColumnReferences(
+                rexBuilder, viewPreds[1], tableMapping.inverse(), qEC.getEquivalenceClassesMap());
+            compensationRangePred = SubstitutionVisitor.splitFilter(
+                simplify, queryRangePred, viewRangePred);
+            if (compensationRangePred == null) {
+              // Skip it
+              continue;
+            }
+
+            // 4.3. Finally, we check that residual predicates of the query are satisfied
+            // within the view.
+            // Compute compensating predicates.
+            final RexNode queryResidualPred = RexUtil.swapColumnReferences(
+                rexBuilder, queryPreds[2], qEC.getEquivalenceClassesMap());
+            final RexNode viewResidualPred = RexUtil.swapTableColumnReferences(
+                rexBuilder, viewPreds[2], tableMapping.inverse(), qEC.getEquivalenceClassesMap());
+            compensationResidualPred = SubstitutionVisitor.splitFilter(
+                simplify, queryResidualPred, viewResidualPred);
+            if (compensationResidualPred == null) {
+              // Skip it
+              continue;
+            }
+
+            // 4.4. Final compensation predicate.
+            RexNode compensationPred = RexUtil.composeConjunction(
+                rexBuilder,
+                ImmutableList.of(
+                    compensationColumnsEquiPred,
+                    compensationRangePred,
+                    compensationResidualPred),
+                false);
+            if (!compensationPred.isAlwaysTrue()) {
+              // All columns required by compensating predicates must be contained
+              // in the view output (condition 2).
+              List<RexNode> viewExprs = extractExpressions(topViewProject, viewNode,
+                  rexBuilder);
+              compensationPred = rewriteExpression(rexBuilder, viewNode, viewExprs,
+                  compensationPred, tableMapping, qEC.getEquivalenceClassesMap(), mq);
+              if (compensationPred == null) {
+                // Skip it
+                continue;
+              }
+            }
+
+            // 4.5. Generate final rewriting if possible.
+            // First, we add the compensation predicate (if any) on top of the view.
+            // Then, we trigger the Aggregate unifying method. This method will either create
+            // a Project or an Aggregate operator on top of the view. It will also compute the
+            // output expressions for the query.
+            RelBuilder builder = call.builder();
+            builder.push(materialization.tableRel);
+            if (!compensationPred.isAlwaysTrue()) {
+              builder.filter(compensationPred);
+            }
+            RelNode result = unify(rexBuilder, builder, builder.build(),
+                topProject, node, topViewProject, viewNode, tableMapping,
+                qEC.getEquivalenceClassesMap(), mq);
+            if (result == null) {
+              // Skip it
+              continue;
+            }
+            call.transformTo(result);
+          }
+        }
+      }
+    }
+  }
+
+  protected abstract boolean isValidPlan(Project topProject, RelNode node,
+      RelMetadataQuery mq);
+
+  protected abstract boolean isViewMatching(RelNode node);
+
+  protected abstract List<RexNode> extractExpressions(Project topProject,
+      RelNode node, RexBuilder rexBuilder);
+
+  /* This method is responsible for rewriting the query using the given view query.
+   *
+   * The input node is a Scan on the view table and possibly a compensation Filter
+   * on top. If a rewriting can be produced, we return that rewriting. If it cannot
+   * be produced, we will return null. */
+  protected abstract RelNode unify(RexBuilder rexBuilder, RelBuilder relBuilder,
+      RelNode input, Project topProject, RelNode node,
+      Project topViewProject, RelNode viewNode,
+      BiMap<RelTableRef, RelTableRef> tableMapping,
+      Map<RexTableInputRef, Set<RexTableInputRef>> equivalenceClassesMap,
+      RelMetadataQuery mq);
+
+  //~ Instances Join ---------------------------------------------------------
+
+  /** Materialized view rewriting for join */
+  private abstract static class MaterializedViewJoinRule
+          extends AbstractMaterializedViewRule {
+    /** Creates a MaterializedViewJoinRule. */
+    protected MaterializedViewJoinRule(RelOptRuleOperand operand,
+        RelBuilderFactory relBuilderFactory, String description) {
+      super(operand, relBuilderFactory, description);
+    }
+
+    @Override protected boolean isValidPlan(Project topProject, RelNode node,
+        RelMetadataQuery mq) {
+      Join join = (Join) node;
+      if (join.getJoinType() != JoinRelType.INNER) {
+        // TODO: Rewriting for non-inner joins not supported yet
+        return false;
+      }
+      return isValidRexNodePlan(join, mq);
+    }
+
+    @Override protected boolean isViewMatching(RelNode node) {
+      if (node instanceof Join) {
+        return true;
+      }
+      if (node instanceof Project && ((Project) node).getInput() instanceof Join) {
+        return true;
+      }
+      return false;
+    }
+
+    @Override protected List<RexNode> extractExpressions(Project topProject,
+        RelNode node, RexBuilder rexBuilder) {
+      Join viewJoin = (Join) node;
+      List<RexNode> viewExprs = new ArrayList<>();
+      if (topProject != null) {
+        for (RexNode e : topProject.getChildExps()) {
+          viewExprs.add(e);
+        }
+      } else {
+        for (int i = 0; i < viewJoin.getRowType().getFieldCount(); i++) {
+          viewExprs.add(rexBuilder.makeInputRef(viewJoin, i));
+        }
+      }
+      return viewExprs;
+    }
+
+    @Override protected RelNode unify(
+        RexBuilder rexBuilder,
+        RelBuilder relBuilder,
+        RelNode input,
+        Project topProject,
+        RelNode node,
+        Project topViewProject,
+        RelNode viewNode,
+        BiMap<RelTableRef, RelTableRef> tableMapping,
+        Map<RexTableInputRef, Set<RexTableInputRef>> equivalenceClassesMap,
+        RelMetadataQuery mq) {
+      List<RexNode> exprs = extractExpressions(topProject, node, rexBuilder);
+      List<RexNode> exprsLineage = new ArrayList<>(exprs.size());
+      for (RexNode expr : exprs) {
+        Set<RexNode> s = mq.getExpressionLineage(node, expr);
+        if (s == null) {
+          // Bail out
+          return null;
+        }
+        assert s.size() == 1;
+        exprsLineage.add(s.iterator().next());
+      }
+      List<RexNode> viewExprs = extractExpressions(topViewProject, viewNode, rexBuilder);
+      List<RexNode> rewrittenExprs = rewriteExpressions(rexBuilder, viewNode, viewExprs,
+          exprsLineage, tableMapping, equivalenceClassesMap, mq);
+      if (rewrittenExprs == null) {
+        return null;
+      }
+      return relBuilder
+          .push(input)
+          .project(rewrittenExprs)
+          .build();
+    }
+  }
+
+  /** Rule that matches Project on Join. */
+  public static class MaterializedViewProjectJoinRule extends MaterializedViewJoinRule {
+    public MaterializedViewProjectJoinRule(RelBuilderFactory relBuilderFactory) {
+      super(
+          operand(Project.class,
+              operand(Join.class, any())),
+          relBuilderFactory,
+          "MaterializedViewJoinRule(Project-Join)");
+    }
+
+    @Override public void onMatch(RelOptRuleCall call) {
+      final Project project = call.rel(0);
+      final Join join = call.rel(1);
+      perform(call, project, join);
+    }
+  }
+
+  /** Rule that matches Join. */
+  public static class MaterializedViewOnlyJoinRule extends MaterializedViewJoinRule {
+    public MaterializedViewOnlyJoinRule(RelBuilderFactory relBuilderFactory) {
+      super(
+          operand(Join.class, any()),
+          relBuilderFactory,
+          "MaterializedViewJoinRule(Join)");
+    }
+
+    @Override public void onMatch(RelOptRuleCall call) {
+      final Join join = call.rel(0);
+      perform(call, null, join);
+    }
+  }
+
+  //~ Instances Aggregate ----------------------------------------------------
+
+  /** Materialized view rewriting for aggregate */
+  private abstract static class MaterializedViewAggregateRule
+          extends AbstractMaterializedViewRule {
+    /** Creates a MaterializedViewAggregateRule. */
+    protected MaterializedViewAggregateRule(RelOptRuleOperand operand,
+        RelBuilderFactory relBuilderFactory, String description) {
+      super(operand, relBuilderFactory, description);
+    }
+
+    @Override protected boolean isValidPlan(Project topProject, RelNode node,
+        RelMetadataQuery mq) {
+      Aggregate aggregate = (Aggregate) node;
+      if (aggregate.getGroupType() != Aggregate.Group.SIMPLE) {
+        // TODO: Rewriting with grouping sets not supported yet
+        return false;
+      }
+      return isValidRexNodePlan(aggregate.getInput(), mq);
+    }
+
+    @Override protected boolean isViewMatching(RelNode node) {
+      if (node instanceof Aggregate) {
+        return true;
+      }
+      if (node instanceof Project && ((Project) node).getInput() instanceof Aggregate) {
+        return true;
+      }
+      return false;
+    }
+
+    @Override protected List<RexNode> extractExpressions(Project topProject,
+        RelNode node, RexBuilder rexBuilder) {
+      Aggregate viewAggregate = (Aggregate) node;
+      List<RexNode> viewExprs = new ArrayList<>();
+      if (topProject != null) {
+        for (RexNode e : topProject.getChildExps()) {
+          viewExprs.add(e);
+        }
+      } else {
+        for (int i = 0; i < viewAggregate.getGroupCount(); i++) {
+          viewExprs.add(rexBuilder.makeInputRef(viewAggregate, i));
+        }
+      }
+      return viewExprs;
+    }
+
+    @Override protected RelNode unify(
+        RexBuilder rexBuilder,
+        RelBuilder relBuilder,
+        RelNode input,
+        Project topProject,
+        RelNode node,
+        Project topViewProject,
+        RelNode viewNode,
+        BiMap<RelTableRef, RelTableRef> tableMapping,
+        Map<RexTableInputRef, Set<RexTableInputRef>> equivalenceClassesMap,
+        RelMetadataQuery mq) {
+      final Aggregate queryAggregate = (Aggregate) node;
+      final Aggregate viewAggregate = (Aggregate) viewNode;
+      // Get group by references and aggregate call input references needed
+      ImmutableBitSet.Builder indexes = ImmutableBitSet.builder();
+      ImmutableBitSet references = null;
+      if (topProject != null) {
+        // We have a Project on top, gather only what is needed
+        final RelOptUtil.InputFinder inputFinder =
+            new RelOptUtil.InputFinder(new LinkedHashSet<RelDataTypeField>());
+        for (RexNode e : topProject.getChildExps()) {
+          e.accept(inputFinder);
+        }
+        references = inputFinder.inputBitSet.build();
+        for (int i = 0; i < queryAggregate.getGroupCount(); i++) {
+          indexes.set(queryAggregate.getGroupSet().nth(i));
+        }
+        for (int i = 0; i < queryAggregate.getAggCallList().size(); i++) {
+          if (references.get(queryAggregate.getGroupCount() + i)) {
+            for (int inputIdx : queryAggregate.getAggCallList().get(i).getArgList()) {
+              indexes.set(inputIdx);
+            }
+          }
+        }
+      } else {
+        // No project on top, all of them are needed
+        for (int i = 0; i < queryAggregate.getGroupCount(); i++) {
+          indexes.set(queryAggregate.getGroupSet().nth(i));
+        }
+        for (AggregateCall queryAggCall : queryAggregate.getAggCallList()) {
+          for (int inputIdx : queryAggCall.getArgList()) {
+            indexes.set(inputIdx);
+          }
+        }
+      }
+
+      // Create mapping from query columns to view columns
+      Multimap<Integer, Integer> m = generateMapping(rexBuilder, queryAggregate.getInput(),
+          viewAggregate.getInput(), indexes.build(), tableMapping, equivalenceClassesMap, mq);
+      if (m == null) {
+        // Bail out
+        return null;
+      }
+
+      // We could map all expressions. Create aggregate mapping.
+      Mapping aggregateMapping = Mappings.create(MappingType.FUNCTION,
+          queryAggregate.getRowType().getFieldCount(), viewAggregate.getRowType().getFieldCount());
+      for (int i = 0; i < queryAggregate.getGroupCount(); i++) {
+        Collection<Integer> c = m.get(queryAggregate.getGroupSet().nth(i));
+        for (int j : c) {
+          int targetIdx = viewAggregate.getGroupSet().indexOf(j);
+          if (targetIdx == -1) {
+            continue;
+          }
+          aggregateMapping.set(i, targetIdx);
+          break;
+        }
+        if (aggregateMapping.getTargetOpt(i) == -1) {
+          // It is not part of group by, we bail out
+          return null;
+        }
+      }
+      for (int idx = 0; idx < queryAggregate.getAggCallList().size(); idx++) {
+        if (references != null && !references.get(queryAggregate.getGroupCount() + idx)) {
+          // Ignore
+          continue;
+        }
+        AggregateCall queryAggCall = queryAggregate.getAggCallList().get(idx);
+        List<Integer> queryAggCallIndexes = queryAggCall.getArgList();
+        for (int j = 0; j < viewAggregate.getAggCallList().size(); j++) {
+          AggregateCall viewAggCall = viewAggregate.getAggCallList().get(j);
+          if (queryAggCall.getAggregation() != viewAggCall.getAggregation()
+              || queryAggCall.isDistinct() != viewAggCall.isDistinct()
+              || queryAggCall.getArgList().size() != viewAggCall.getArgList().size()
+              || queryAggCall.getType() != viewAggCall.getType()) {
+            // Continue
+            continue;
+          }
+          List<Integer> viewAggCallIndexes = new ArrayList<>();
+          for (int aggCallIdx : viewAggCall.getArgList()) {
+            viewAggCallIndexes.add(m.get(aggCallIdx).iterator().next());
+          }
+          if (!queryAggCallIndexes.equals(viewAggCallIndexes)) {
+            // Continue
+            continue;
+          }
+          aggregateMapping.set(queryAggregate.getGroupCount() + idx,
+              viewAggregate.getGroupCount() + j);
+          break;
+        }
+      }
+
+      // Generate result rewriting
+      Mapping rewritingMapping = null;
+      RelNode result = relBuilder
+          .push(input)
+          .build();
+      if (queryAggregate.getGroupCount() != viewAggregate.getGroupCount()) {
+        // Target is coarser level of aggregation. Generate an aggregate.
+        rewritingMapping = Mappings.create(MappingType.FUNCTION,
+            topViewProject != null ? topViewProject.getRowType().getFieldCount()
+                : viewAggregate.getRowType().getFieldCount(),
+            queryAggregate.getRowType().getFieldCount());
+        final ImmutableBitSet.Builder groupSetB = ImmutableBitSet.builder();
+        for (int i = 0; i < queryAggregate.getGroupCount(); i++) {
+          int targetIdx = aggregateMapping.getTargetOpt(i);
+          if (targetIdx == -1) {
+            // No matching group by column, we bail out
+            return null;
+          }
+          if (topViewProject != null) {
+            boolean added = false;
+            for (int k = 0; k < topViewProject.getChildExps().size(); k++) {
+              RexNode n = topViewProject.getChildExps().get(k);
+              if (!n.isA(SqlKind.INPUT_REF)) {
+                continue;
+              }
+              int ref = ((RexInputRef) n).getIndex();
+              if (ref == targetIdx) {
+                groupSetB.set(k);
+                rewritingMapping.set(k, i);
+                added = true;
+                break;
+              }
+            }
+            if (!added) {
+              // No matching group by column, we bail out
+              return null;
+            }
+          } else {
+            groupSetB.set(targetIdx);
+            rewritingMapping.set(targetIdx, i);
+          }
+        }
+        final ImmutableBitSet groupSet = groupSetB.build();
+        final List<AggCall> aggregateCalls = new ArrayList<>();
+        for (int i = 0; i < queryAggregate.getAggCallList().size(); i++) {
+          if (references != null && !references.get(queryAggregate.getGroupCount() + i)) {
+            // Ignore
+            continue;
+          }
+          int sourceIdx = queryAggregate.getGroupCount() + i;
+          int targetIdx = aggregateMapping.getTargetOpt(sourceIdx);
+          if (targetIdx == -1) {
+            // No matching aggregation column, we bail out
+            return null;
+          }
+          AggregateCall queryAggCall = queryAggregate.getAggCallList().get(i);
+          if (topViewProject != null) {
+            boolean added = false;
+            for (int k = 0; k < topViewProject.getChildExps().size(); k++) {
+              RexNode n = topViewProject.getChildExps().get(k);
+              if (!n.isA(SqlKind.INPUT_REF)) {
+                continue;
+              }
+              int ref = ((RexInputRef) n).getIndex();
+              if (ref == targetIdx) {
+                aggregateCalls.add(
+                    relBuilder.aggregateCall(
+                        SubstitutionVisitor.getRollup(queryAggCall.getAggregation()),
+                        queryAggCall.isDistinct(),
+                        null,
+                        queryAggCall.name,
+                        ImmutableList.of(rexBuilder.makeInputRef(input, k))));
+                rewritingMapping.set(k, sourceIdx);
+                added = true;
+                break;
+              }
+            }
+            if (!added) {
+              // No matching aggregation column, we bail out
+              return null;
+            }
+          } else {
+            aggregateCalls.add(
+                relBuilder.aggregateCall(
+                    SubstitutionVisitor.getRollup(queryAggCall.getAggregation()),
+                    queryAggCall.isDistinct(),
+                    null,
+                    queryAggCall.name,
+                    ImmutableList.of(rexBuilder.makeInputRef(input, targetIdx))));
+            rewritingMapping.set(targetIdx, sourceIdx);
+          }
+        }
+        result = relBuilder
+            .push(result)
+            .aggregate(relBuilder.groupKey(groupSet, false, null), aggregateCalls)
+            .build();
+        // We introduce a project on top, as group by columns order is lost
+        List<RexNode> projects = new ArrayList<>();
+        Mapping inverseMapping = rewritingMapping.inverse();
+        for (int i = 0; i < queryAggregate.getGroupCount(); i++) {
+          projects.add(
+              rexBuilder.makeInputRef(result,
+                  groupSet.indexOf(inverseMapping.getTarget(i))));
+        }
+        for (int i = 0; i < queryAggregate.getAggCallList().size(); i++) {
+          projects.add(
+              rexBuilder.makeInputRef(result, queryAggregate.getGroupCount() + i));
+        }
+        result = relBuilder
+            .push(result)
+            .project(projects)
+            .build();
+      } // end if queryAggregate.getGroupCount() != viewAggregate.getGroupCount()
+
+      // Add query expressions on top. We first map query expressions to view
+      // expressions. Once we have done that, if the expression is contained
+      // and we have introduced already an operator on top of the input node,
+      // we use the mapping to resolve the position of the expression in the
+      // node.
+      final List<RexNode> topExprs = new ArrayList<>();
+      if (topProject != null) {
+        topExprs.addAll(topProject.getChildExps());
+      } else {
+        // Add all
+        for (int pos = 0; pos < queryAggregate.getRowType().getFieldCount(); pos++) {
+          topExprs.add(rexBuilder.makeInputRef(queryAggregate, pos));
+        }
+      }
+      // Available in view.
+      final List<String> viewExprs = new ArrayList<>();
+      if (topViewProject != null) {
+        for (int i = 0; i < topViewProject.getChildExps().size(); i++) {
+          viewExprs.add(topViewProject.getChildExps().get(i).toString());
+        }
+      } else {
+        // Add all
+        for (int i = 0; i < viewAggregate.getRowType().getFieldCount(); i++) {
+          viewExprs.add(rexBuilder.makeInputRef(viewAggregate, i).toString());
+        }
+      }
+      final List<RexNode> rewrittenExprs = new ArrayList<>(topExprs.size());
+      for (RexNode expr : topExprs) {
+        RexNode rewrittenExpr = shuttleReferences(rexBuilder, expr, aggregateMapping);
+        if (rewrittenExpr == null) {
+          // Cannot map expression
+          return null;
+        }
+        int pos = viewExprs.indexOf(rewrittenExpr.toString());
+        if (pos == -1) {
+          // Cannot map expression
+          return null;
+        }
+        if (rewritingMapping != null) {
+          pos = rewritingMapping.getTargetOpt(pos);
+          if (pos == -1) {
+            // Cannot map expression
+            return null;
+          }
+        }
+        rewrittenExprs.add(rexBuilder.makeInputRef(result, pos));
+      }
+      return relBuilder
+          .push(result)
+          .project(rewrittenExprs)
+          .build();
+    }
+  }
+
+  /** Rule that matches Project on Aggregate. */
+  public static class MaterializedViewProjectAggregateRule extends MaterializedViewAggregateRule {
+    public MaterializedViewProjectAggregateRule(RelBuilderFactory relBuilderFactory) {
+      super(
+          operand(Project.class,
+              operand(Aggregate.class, any())),
+          relBuilderFactory,
+          "MaterializedViewAggregateRule(Project-Aggregate)");
+    }
+
+    @Override public void onMatch(RelOptRuleCall call) {
+      final Project project = call.rel(0);
+      final Aggregate aggregate = call.rel(1);
+      perform(call, project, aggregate);
+    }
+  }
+
+  /** Rule that matches Aggregate. */
+  public static class MaterializedViewOnlyAggregateRule extends MaterializedViewAggregateRule {
+    public MaterializedViewOnlyAggregateRule(RelBuilderFactory relBuilderFactory) {
+      super(
+          operand(Aggregate.class, any()),
+          relBuilderFactory,
+          "MaterializedViewAggregateRule(Aggregate)");
+    }
+
+    @Override public void onMatch(RelOptRuleCall call) {
+      final Aggregate aggregate = call.rel(0);
+      perform(call, null, aggregate);
+    }
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  /* It will flatten a multimap containing table references to table references,
+   * producing all possible combinations of mappings. Each of the mappings will
+   * be bi-directional. */
+  private static List<BiMap<RelTableRef, RelTableRef>> generateTableMappings(
+      Multimap<RelTableRef, RelTableRef> multiMapTables) {
+    final List<BiMap<RelTableRef, RelTableRef>> result = new ArrayList<>();
+    if (multiMapTables.isEmpty()) {
+      return result;
+    }
+    result.add(HashBiMap.<RelTableRef, RelTableRef>create());
+    for (Entry<RelTableRef, Collection<RelTableRef>> e : multiMapTables.asMap().entrySet()) {
+      boolean added = false;
+      for (RelTableRef target : e.getValue()) {
+        if (added) {
+          for (BiMap<RelTableRef, RelTableRef> m : result) {
+            final BiMap<RelTableRef, RelTableRef> newM =
+                HashBiMap.<RelTableRef, RelTableRef>create(m);
+            newM.put(e.getKey(), target);
+            result.add(newM);
+          }
+        } else {
+          for (BiMap<RelTableRef, RelTableRef> m : result) {
+            m.put(e.getKey(), target);
+          }
+          added = true;
+        }
+      }
+      // Mapping needs to exist
+      assert added;
+    }
+    return result;
+  }
+
+  /* Currently we only support TableScan - Project - Filter - Join */
+  private static boolean isValidRexNodePlan(RelNode node, RelMetadataQuery mq) {
+    final Multimap<Class<? extends RelNode>, RelNode> m =
+            mq.getNodeTypes(node);
+    for (Class<? extends RelNode> c : m.keySet()) {
+      if (!TableScan.class.isAssignableFrom(c)
+              && !Project.class.isAssignableFrom(c)
+              && !Filter.class.isAssignableFrom(c)
+              && !Join.class.isAssignableFrom(c)) {
+        // Skip it
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /* Classifies each of the predicates in the list into one of these three
+   * categories:
+   * - column equality predicates, or
+   * - range predicates, comprising <, <=, >, >=, and = between a reference
+   * and a constant, or
+   * - residual predicates, all the rest
+   *
+   * For each category, it creates the conjunction of the predicates. The
+   * result is an array of three RexNode objects corresponding to each
+   * category. */
+  private static RexNode[] splitPredicates(
+      RexBuilder rexBuilder, ImmutableList<RexNode> predicates) {
+    List<RexNode> equiColumnsPreds = new ArrayList<>();
+    List<RexNode> rangePreds = new ArrayList<>();
+    List<RexNode> residualPreds = new ArrayList<>();
+    for (RexNode e : predicates) {
+      switch (e.getKind()) {
+      case EQUALS:
+        RexCall eqCall = (RexCall) e;
+        if (RexUtil.isReferenceOrAccess(eqCall.getOperands().get(0), false)
+                && RexUtil.isReferenceOrAccess(eqCall.getOperands().get(1), false)) {
+          equiColumnsPreds.add(e);
+        } else if ((RexUtil.isReferenceOrAccess(eqCall.getOperands().get(0), false)
+                && RexUtil.isConstant(eqCall.getOperands().get(1)))
+            || (RexUtil.isReferenceOrAccess(eqCall.getOperands().get(1), false)
+                && RexUtil.isConstant(eqCall.getOperands().get(0)))) {
+          rangePreds.add(e);
+        } else {
+          residualPreds.add(e);
+        }
+        break;
+      case LESS_THAN:
+      case GREATER_THAN:
+      case LESS_THAN_OR_EQUAL:
+      case GREATER_THAN_OR_EQUAL:
+      case NOT_EQUALS:
+        RexCall rangeCall = (RexCall) e;
+        if ((RexUtil.isReferenceOrAccess(rangeCall.getOperands().get(0), false)
+                && RexUtil.isConstant(rangeCall.getOperands().get(1)))
+            || (RexUtil.isReferenceOrAccess(rangeCall.getOperands().get(1), false)
+                && RexUtil.isConstant(rangeCall.getOperands().get(0)))) {
+          rangePreds.add(e);
+        } else {
+          residualPreds.add(e);
+        }
+        break;
+      default:
+        residualPreds.add(e);
+      }
+    }
+    return new RexNode[] {
+        RexUtil.composeConjunction(rexBuilder, equiColumnsPreds, false),
+        RexUtil.composeConjunction(rexBuilder, rangePreds, false),
+        RexUtil.composeConjunction(rexBuilder, residualPreds, false)};
+  }
+
+  /* Given the equi-column predicates of the query and the view and the
+   * computed equivalence classes, it extracts possible mappings between
+   * the equivalence classes.
+   *
+   * If there is no mapping, it returns null. If there is a exact match,
+   * it will return a compensation predicate that evaluates to true.
+   * Finally, if a compensation predicate needs to be enforced on top of
+   * the view to make the equivalences classes match, it returns that
+   * compensation predicate */
+  private static RexNode generateEquivalenceClasses(RexBuilder rexBuilder,
+      RexNode queryEquiColumnsPreds, EquivalenceClasses qEC,
+      RexNode viewEquiColumnsPreds, EquivalenceClasses vEC) {
+    if (queryEquiColumnsPreds.isAlwaysTrue() && viewEquiColumnsPreds.isAlwaysTrue()) {
+      // No column equality predicates in query and view
+      // Empty mapping and compensation predicate
+      assert qEC.getEquivalenceClasses().isEmpty() && vEC.getEquivalenceClasses().isEmpty();
+      return rexBuilder.makeLiteral(true);
+    }
+    if (queryEquiColumnsPreds.isAlwaysTrue() || viewEquiColumnsPreds.isAlwaysTrue()) {
+      // No column equality predicates in query or view
+      assert qEC.getEquivalenceClasses().isEmpty() || vEC.getEquivalenceClasses().isEmpty();
+      return null;
+    }
+
+    final List<Set<RexTableInputRef>> queryEquivalenceClasses = qEC.getEquivalenceClasses();
+    final List<Set<RexTableInputRef>> viewEquivalenceClasses = vEC.getEquivalenceClasses();
+    final Mapping mapping = extractPossibleMapping(
+        queryEquivalenceClasses, viewEquivalenceClasses);
+    if (mapping == null) {
+      // Did not find mapping between the equivalence classes,
+      // bail out
+      return null;
+    }
+
+    // Create the compensation predicate
+    RexNode compensationPredicate = rexBuilder.makeLiteral(true);
+    for (IntPair pair : mapping) {
+      Set<RexTableInputRef> difference = new HashSet<>(
+          queryEquivalenceClasses.get(pair.target));
+      difference.removeAll(viewEquivalenceClasses.get(pair.source));
+      for (RexTableInputRef e : difference) {
+        RexNode equals = rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+            e, viewEquivalenceClasses.get(pair.source).iterator().next());
+        compensationPredicate = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+            compensationPredicate, equals);
+      }
+    }
+
+    return compensationPredicate;
+  }
+
+  /* Given the query and view equivalence classes, it extracts the possible mappings
+   * from each view equivalence class to each query equivalence class.
+   *
+   * If any of the view equivalence classes cannot be mapped to a query equivalence
+   * class, it returns null. */
+  private static Mapping extractPossibleMapping(
+      List<Set<RexTableInputRef>> queryEquivalenceClasses,
+      List<Set<RexTableInputRef>> viewEquivalenceClasses) {
+    Mapping mapping = Mappings.create(MappingType.FUNCTION,
+        viewEquivalenceClasses.size(), queryEquivalenceClasses.size());
+    for (int i = 0; i < viewEquivalenceClasses.size(); i++) {
+      boolean foundQueryEquivalenceClass = false;
+      final Set<RexTableInputRef> viewEquivalenceClass = viewEquivalenceClasses.get(i);
+      for (int j = 0; j < queryEquivalenceClasses.size(); j++) {
+        final Set<RexTableInputRef> queryEquivalenceClass = queryEquivalenceClasses.get(j);
+        if (queryEquivalenceClass.containsAll(viewEquivalenceClass)) {
+          mapping.set(i, j);
+          foundQueryEquivalenceClass = true;
+          break;
+        }
+      } // end for
+
+      if (!foundQueryEquivalenceClass) {
+        // View equivalence class not found in query equivalence class
+        return null;
+      }
+    } // end for
+
+    return mapping;
+  }
+
+  /* Given the input expression that references source expressions in the query,
+   * it will rewrite it to refer to the view output.
+   *
+   * If any of the subexpressions in the input expression cannot be mapped to
+   * the query, it will return null.
+   */
+  private static RexNode rewriteExpression(
+      RexBuilder rexBuilder,
+      RelNode viewNode,
+      List<RexNode> viewExprs,
+      RexNode expr,
+      BiMap<RelTableRef, RelTableRef> tableMapping,
+      Map<RexTableInputRef, Set<RexTableInputRef>> equivalenceClassesMap,
+      RelMetadataQuery mq) {
+    List<RexNode> rewrittenExprs = rewriteExpressions(rexBuilder, viewNode, viewExprs,
+        ImmutableList.of(expr), tableMapping, equivalenceClassesMap, mq);
+    if (rewrittenExprs == null) {
+      return null;
+    }
+    assert rewrittenExprs.size() == 1;
+    return rewrittenExprs.get(0);
+  }
+
+  private static List<RexNode> rewriteExpressions(
+      RexBuilder rexBuilder,
+      RelNode viewNode,
+      List<RexNode> viewExprs,
+      List<RexNode> exprs,
+      BiMap<RelTableRef, RelTableRef> tableMapping,
+      Map<RexTableInputRef, Set<RexTableInputRef>> equivalenceClassesMap,
+      RelMetadataQuery mq) {
+    Map<String, Integer> exprsLineage = new HashMap<>();
+    for (int i = 0; i < viewExprs.size(); i++) {
+      final RexNode e = viewExprs.get(i);
+      final Set<RexNode> s = mq.getExpressionLineage(viewNode, e);
+      if (s == null) {
+        // Next expression
+        continue;
+      }
+      // We only support project - filter - join, thus it should map to
+      // a single expression
+      assert s.size() == 1;
+      // Rewrite expr to be expressed on query tables
+      exprsLineage.put(
+          RexUtil.swapTableColumnReferences(
+              rexBuilder,
+              s.iterator().next(),
+              tableMapping.inverse(),
+              equivalenceClassesMap).toString(),
+          i);
+    }
+
+    List<RexNode> rewrittenExprs = new ArrayList<>(exprs.size());
+    for (RexNode expr : exprs) {
+      RexNode rewrittenExpr = replaceWithOriginalReferences(rexBuilder, expr, exprsLineage);
+      if (RexUtil.containsTableInputRef(rewrittenExpr) != null) {
+        // Some expressions were not present in view output
+        return null;
+      }
+      rewrittenExprs.add(rewrittenExpr);
+    }
+    return rewrittenExprs;
+  }
+
+  /* Mapping from node expressions to target expressions.
+   *
+   * If any of the expressions cannot be mapped, we return null. */
+  private static Multimap<Integer, Integer> generateMapping(
+      RexBuilder rexBuilder,
+      RelNode node,
+      RelNode target,
+      ImmutableBitSet positions,
+      BiMap<RelTableRef, RelTableRef> tableMapping,
+      Map<RexTableInputRef, Set<RexTableInputRef>> equivalenceClassesMap,
+      RelMetadataQuery mq) {
+    Multimap<String, Integer> exprsLineage = ArrayListMultimap.create();
+    for (int i = 0; i < target.getRowType().getFieldCount(); i++) {
+      Set<RexNode> s = mq.getExpressionLineage(target, rexBuilder.makeInputRef(target, i));
+      if (s == null) {
+        // Bail out
+        continue;
+      }
+      // We only support project - filter - join, thus it should map to
+      // a single expression
+      assert s.size() == 1;
+      // Rewrite expr to be expressed on query tables
+      exprsLineage.put(
+          RexUtil.swapTableColumnReferences(
+              rexBuilder,
+              s.iterator().next(),
+              tableMapping.inverse(),
+              equivalenceClassesMap).toString(),
+          i);
+    }
+
+    Multimap<Integer, Integer> m = ArrayListMultimap.create();
+    for (int i : positions) {
+      Set<RexNode> s = mq.getExpressionLineage(node, rexBuilder.makeInputRef(node, i));
+      if (s == null) {
+        // Bail out
+        return null;
+      }
+      // We only support project - filter - join, thus it should map to
+      // a single expression
+      assert s.size() == 1;
+      // Rewrite expr to be expressed on query tables
+      Collection<Integer> c = exprsLineage.get(
+          RexUtil.swapColumnReferences(
+              rexBuilder, s.iterator().next(), equivalenceClassesMap).toString());
+      if (c == null) {
+        // Bail out
+        return null;
+      }
+      for (Integer j : c) {
+        m.put(i, j);
+      }
+    }
+    return m;
+  }
+
+  /* Given the input expression, it will replace (sub)expressions when possible
+   * using the content of the mapping. In particular, the mapping contains the
+   * digest of the expression and the index that the replacement input ref should
+   * point to. */
+  private static RexNode replaceWithOriginalReferences(final RexBuilder rexBuilder,
+      final RexNode expr, final Map<String, Integer> mapping) {
+    // Currently we allow the following:
+    // 1) compensation pred can be directly map to expression
+    // 2) all references in compensation pred can be map to expressions
+    RexShuttle visitor =
+        new RexShuttle() {
+          @Override public RexNode visitCall(RexCall call) {
+            Integer pos = mapping.get(call.toString());
+            if (pos != null) {
+              // Found it
+              return rexBuilder.makeInputRef(call.getType(), pos);
+            }
+            return super.visitCall(call);
+          }
+
+          @Override public RexNode visitTableInputRef(RexTableInputRef inputRef) {
+            Integer pos = mapping.get(inputRef.toString());
+            if (pos != null) {
+              // Found it
+              return rexBuilder.makeInputRef(inputRef.getType(), pos);
+            }
+            return super.visitTableInputRef(inputRef);
+          }
+        };
+    return visitor.apply(expr);
+  }
+
+  /* Replaces all the input references by the position in the
+   * input column set. If a reference index cannot be found in
+   * the input set, then we return null. */
+  private static RexNode shuttleReferences(final RexBuilder rexBuilder,
+      final RexNode node, final Mapping mapping) {
+    try {
+      RexShuttle visitor =
+          new RexShuttle() {
+            @Override public RexNode visitInputRef(RexInputRef inputRef) {
+              int pos = mapping.getTargetOpt(inputRef.getIndex());
+              if (pos != -1) {
+                // Found it
+                return rexBuilder.makeInputRef(inputRef.getType(), pos);
+              }
+              throw Util.FoundOne.NULL;
+            }
+          };
+      return visitor.apply(node);
+    } catch (Util.FoundOne ex) {
+      Util.swallow(ex, null);
+      return null;
+    }
+  }
+
+  /**
+   * Class representing an equivalence class, i.e., a set of equivalent columns.
+   */
+  private static class EquivalenceClasses {
+
+    private Map<RexTableInputRef, Set<RexTableInputRef>> nodeToEquivalenceClass;
+
+    protected EquivalenceClasses() {
+      nodeToEquivalenceClass = new HashMap<>();
+    }
+
+    protected void addEquivalenceClass(RexTableInputRef p1, RexTableInputRef p2) {
+      Set<RexTableInputRef> c1 = nodeToEquivalenceClass.get(p1);
+      Set<RexTableInputRef> c2 = nodeToEquivalenceClass.get(p2);
+      if (c1 != null && c2 != null) {
+        // Both present, we need to merge
+        if (c1.size() < c2.size()) {
+          // We swap them to merge
+          c1 = c2;
+          p1 = p2;
+        }
+        for (RexTableInputRef newRef : c2) {
+          c1.add(newRef);
+          nodeToEquivalenceClass.put(newRef, c1);
+        }
+      } else if (c1 != null) {
+        // p1 present, we need to merge into it
+        c1.add(p2);
+        nodeToEquivalenceClass.put(p2, c1);
+      } else if (c2 != null) {
+        // p2 present, we need to merge into it
+        c2.add(p1);
+        nodeToEquivalenceClass.put(p1, c2);
+      } else {
+        // None are present, add to same equivalence class
+        Set<RexTableInputRef> equivalenceClass = new LinkedHashSet<>();
+        equivalenceClass.add(p1);
+        equivalenceClass.add(p2);
+        nodeToEquivalenceClass.put(p1, equivalenceClass);
+        nodeToEquivalenceClass.put(p2, equivalenceClass);
+      }
+    }
+
+    protected Map<RexTableInputRef, Set<RexTableInputRef>> getEquivalenceClassesMap() {
+      return ImmutableMap.copyOf(nodeToEquivalenceClass);
+    }
+
+    protected List<Set<RexTableInputRef>> getEquivalenceClasses() {
+      return ImmutableList.copyOf(nodeToEquivalenceClass.values());
+    }
+  }
+}
+
+// End AbstractMaterializedViewRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
index 7e2d26b..b986684 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
@@ -16,18 +16,35 @@
  */
 package org.apache.calcite.rex;
 
-import org.apache.calcite.rel.metadata.RelTableRef;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
 /**
- * Variable which references a field of an input relational expression
+ * Variable which references a column of a table occurrence in a relational plan.
+ *
+ * <p>This object is used by:
+ * - {@link {@link org.apache.calcite.rel.metadata.BuiltInMetadata.ExpressionLineage}, and
+ * - {@link {@link org.apache.calcite.rel.metadata.BuiltInMetadata.AllPredicates}.
+ *
+ * <p>Given a relational expression, its purpose is to be able to reference uniquely
+ * the provenance of a given expression. For that, it uses a unique table reference
+ * (contained in a {@link RelTableRef}) and an column index within the table.
+ *
+ * <p>For example, {@code A.#0.$3 + 2} column {@code $3} in the {@code 0} occurrence of
+ * table {@code A} in the plan.
+ *
+ * Note that this kind of {@link RexNode} is an auxiliary data structure with a very
+ * specific purpose and should not be used in relational expressions.
  */
 public class RexTableInputRef extends RexInputRef {
 
   private final RelTableRef tableRef;
 
-  public RexTableInputRef(RelTableRef tableRef, int index, RelDataType type) {
+  private RexTableInputRef(RelTableRef tableRef, int index, RelDataType type) {
     super(index, type);
     this.tableRef = tableRef;
     this.digest = tableRef.toString() + ".$" + index;
@@ -50,12 +67,12 @@ public class RexTableInputRef extends RexInputRef {
     return tableRef;
   }
 
-  public String getQualifiedName() {
+  public List<String> getQualifiedName() {
     return tableRef.getQualifiedName();
   }
 
   public int getIdentifier() {
-    return tableRef.getIdentifier();
+    return tableRef.getEntityNumber();
   }
 
   public static RexTableInputRef of(RelTableRef tableRef, int index, RelDataType type) {
@@ -77,6 +94,49 @@ public class RexTableInputRef extends RexInputRef {
   @Override public SqlKind getKind() {
     return SqlKind.TABLE_INPUT_REF;
   }
+
+  /** Identifies uniquely a table by its qualified name and its entity number (occurrence) */
+  public static class RelTableRef {
+
+    private final List<String> qualifiedName;
+    private final int entityNumber;
+    private final String digest;
+
+    private RelTableRef(List<String> qualifiedName, int entityNumber) {
+      this.qualifiedName = ImmutableList.copyOf(qualifiedName);
+      this.entityNumber = entityNumber;
+      this.digest = qualifiedName + ".#" + entityNumber;
+    }
+
+    //~ Methods ----------------------------------------------------------------
+
+    @Override public boolean equals(Object obj) {
+      return this == obj
+          || obj instanceof RelTableRef
+          && qualifiedName.equals(((RelTableRef) obj).qualifiedName)
+          && entityNumber == ((RelTableRef) obj).entityNumber;
+    }
+
+    @Override public int hashCode() {
+      return digest.hashCode();
+    }
+
+    public List<String> getQualifiedName() {
+      return qualifiedName;
+    }
+
+    public int getEntityNumber() {
+      return entityNumber;
+    }
+
+    @Override public String toString() {
+      return digest;
+    }
+
+    public static RelTableRef of(List<String> qualifiedName, int entityNumber) {
+      return new RelTableRef(qualifiedName, entityNumber);
+    }
+  }
 }
 
 // End RexTableInputRef.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 0af3fc7..95ab2c2 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -24,11 +24,11 @@ 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.metadata.RelTableRef;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexTableInputRef.RelTableRef;
 import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.sql.SqlAggFunction;
@@ -1867,16 +1867,31 @@ public class RexUtil {
     }
   }
 
+  /**
+   * Given an expression, it will swap the table references contained in its
+   * {@link RexTableInputRef} using the contents in the map.
+   */
   public static RexNode swapTableReferences(final RexBuilder rexBuilder,
       final RexNode node, final Map<RelTableRef, RelTableRef> tableMapping) {
     return swapTableColumnReferences(rexBuilder, node, tableMapping, null);
   }
 
+  /**
+   * Given an expression, it will swap its column references {@link RexTableInputRef}
+   * using the contents in the map (in particular, the first element of the set in the
+   * map value).
+   */
   public static RexNode swapColumnReferences(final RexBuilder rexBuilder,
       final RexNode node, final Map<RexTableInputRef, Set<RexTableInputRef>> ec) {
     return swapTableColumnReferences(rexBuilder, node, null, ec);
   }
 
+  /**
+   * Given an expression, it will swap the table references contained in its
+   * {@link RexTableInputRef} using the contents in the first map, and then
+   * it will swap the column references {@link RexTableInputRef} using the contents
+   * in the second map (in particular, the first element of the set in the map value).
+   */
   public static RexNode swapTableColumnReferences(final RexBuilder rexBuilder,
       final RexNode node, final Map<RelTableRef, RelTableRef> tableMapping,
       final Map<RexTableInputRef, Set<RexTableInputRef>> ec) {
@@ -1884,7 +1899,7 @@ public class RexUtil {
         new RexShuttle() {
           @Override public RexNode visitTableInputRef(RexTableInputRef inputRef) {
             if (tableMapping != null) {
-              inputRef = new RexTableInputRef(
+              inputRef = RexTableInputRef.of(
                   tableMapping.get(inputRef.getTableRef()),
                   inputRef.getIndex(),
                   inputRef.getType());

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 df2f2f4..3265db5 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -64,6 +64,7 @@ import org.apache.calcite.rel.metadata.BuiltInMetadata.Predicates;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.RowCount;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.Selectivity;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.Size;
+import org.apache.calcite.rel.metadata.BuiltInMetadata.TableReferences;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.UniqueKeys;
 import org.apache.calcite.rel.metadata.Metadata;
 import org.apache.calcite.rex.RexNode;
@@ -376,6 +377,7 @@ public enum BuiltInMethod {
       ImmutableBitSet.class),
   COLUMN_ORIGIN(ColumnOrigin.class, "getColumnOrigins", int.class),
   EXPRESSION_LINEAGE(ExpressionLineage.class, "getExpressionLineage", RexNode.class),
+  TABLE_REFERENCES(TableReferences.class, "getTableReferences"),
   CUMULATIVE_COST(CumulativeCost.class, "getCumulativeCost"),
   NON_CUMULATIVE_COST(NonCumulativeCost.class, "getNonCumulativeCost"),
   PREDICATES(Predicates.class, "getPredicates"),

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 4ad07d4..29603b1 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -60,6 +60,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
 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;
 
@@ -743,6 +744,8 @@ public class MaterializationTest {
         rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, x, i1); // $0 = 1
     final RexNode x_eq_1_b =
         rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, x, i1); // $0 = 1 again
+    final RexNode x_eq_2 =
+        rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, x, i2); // $0 = 2
     final RexNode y_eq_2 =
         rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, y, i2); // $1 = 2
     final RexNode z_eq_3 =
@@ -757,32 +760,32 @@ public class MaterializationTest {
     //   condition: x = 1,
     //   target:    x = 1 or z = 3
     // yields
-    //   residue:   not (z = 3)
+    //   residue:   x = 1
     newFilter = SubstitutionVisitor.splitFilter(simplify,
         x_eq_1,
         rexBuilder.makeCall(SqlStdOperatorTable.OR, x_eq_1, z_eq_3));
-    assertThat(newFilter.toString(), equalTo("NOT(=($2, 3))"));
+    assertThat(newFilter.toString(), equalTo("=($0, 1)"));
 
     // 2b.
     //   condition: x = 1 or y = 2
     //   target:    x = 1 or y = 2 or z = 3
     // yields
-    //   residue:   not (z = 3)
+    //   residue:   x = 1 or y = 2
     newFilter = SubstitutionVisitor.splitFilter(simplify,
         rexBuilder.makeCall(SqlStdOperatorTable.OR, x_eq_1, y_eq_2),
         rexBuilder.makeCall(SqlStdOperatorTable.OR, x_eq_1, y_eq_2, z_eq_3));
-    assertThat(newFilter.toString(), equalTo("NOT(=($2, 3))"));
+    assertThat(newFilter.toString(), equalTo("OR(=($0, 1), =($1, 2))"));
 
     // 2c.
     //   condition: x = 1
     //   target:    x = 1 or y = 2 or z = 3
     // yields
-    //   residue:   not (y = 2) and not (z = 3)
+    //   residue:   x = 1
     newFilter = SubstitutionVisitor.splitFilter(simplify,
         x_eq_1,
         rexBuilder.makeCall(SqlStdOperatorTable.OR, x_eq_1, y_eq_2, z_eq_3));
     assertThat(newFilter.toString(),
-        equalTo("AND(NOT(=($1, 2)), NOT(=($2, 3)))"));
+        equalTo("=($0, 1)"));
 
     // 2d.
     //   condition: x = 1 or y = 2
@@ -807,16 +810,59 @@ public class MaterializationTest {
     //   target:    x = 1
     // yields
     //   residue:   null
-    // TODO:
+    newFilter = SubstitutionVisitor.splitFilter(simplify,
+        rexBuilder.makeCall(SqlStdOperatorTable.OR, x_eq_1, y_eq_2),
+        x_eq_1);
+    assertNull(newFilter);
 
     // Example 3.
     // Condition [x = 1 and y = 2],
     // target [y = 2 and x = 1] yields
     // residue [true].
-    // TODO:
+    newFilter = SubstitutionVisitor.splitFilter(simplify,
+        rexBuilder.makeCall(SqlStdOperatorTable.AND, x_eq_1, y_eq_2),
+        rexBuilder.makeCall(SqlStdOperatorTable.AND, y_eq_2, x_eq_1));
+    assertThat(newFilter.isAlwaysTrue(), equalTo(true));
 
     // Example 4.
-    // TODO:
+    //   condition: x = 1 and y = 2
+    //   target:    y = 2
+    // yields
+    //   residue:   x = 1
+    newFilter = SubstitutionVisitor.splitFilter(simplify,
+        rexBuilder.makeCall(SqlStdOperatorTable.AND, x_eq_1, y_eq_2),
+        y_eq_2);
+    assertThat(newFilter.toString(), equalTo("=($0, 1)"));
+
+    // Example 5.
+    //   condition: x = 1
+    //   target:    x = 1 and y = 2
+    // yields
+    //   residue:   null
+    newFilter = SubstitutionVisitor.splitFilter(simplify,
+        x_eq_1,
+        rexBuilder.makeCall(SqlStdOperatorTable.AND, x_eq_1, y_eq_2));
+    assertNull(newFilter);
+
+    // Example 6.
+    //   condition: x = 1
+    //   target:    y = 2
+    // yields
+    //   residue:   null
+    newFilter = SubstitutionVisitor.splitFilter(simplify,
+        x_eq_1,
+        y_eq_2);
+    assertNull(newFilter);
+
+    // Example 7.
+    //   condition: x = 1
+    //   target:    x = 2
+    // yields
+    //   residue:   null
+    newFilter = SubstitutionVisitor.splitFilter(simplify,
+        x_eq_1,
+        x_eq_2);
+    assertNull(newFilter);
   }
 
   /** Tests a complicated star-join query on a complicated materialized
@@ -924,6 +970,287 @@ public class MaterializationTest {
             "EnumerableTableScan(table=[[hr, m0]])", 1));
   }
 
+  @Test public void testAggregateMaterializationNoAggregateFuncs1() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
+      "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationNoAggregateFuncs2() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
+      "select \"deptno\" from \"emps\" group by \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationNoAggregateFuncs3() {
+    checkNoMaterialize(
+      "select \"deptno\" from \"emps\" group by \"deptno\"",
+      "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
+      JdbcTest.HR_MODEL);
+  }
+
+  @Test public void testAggregateMaterializationNoAggregateFuncs4() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"empid\", \"deptno\"",
+      "select \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationNoAggregateFuncs5() {
+    checkNoMaterialize(
+      "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" = 5 group by \"empid\", \"deptno\"",
+      "select \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"deptno\"",
+      JdbcTest.HR_MODEL);
+  }
+
+  @Test public void testAggregateMaterializationNoAggregateFuncs6() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" > 5 group by \"empid\", \"deptno\"",
+      "select \"deptno\" from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}])\n"
+              + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[10], expr#3=[>($t1, $t2)], "
+              + "proj#0..1=[{exprs}], $condition=[$t3])\n"
+              + "    EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationNoAggregateFuncs7() {
+    checkNoMaterialize(
+      "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" > 5 group by \"empid\", \"deptno\"",
+      "select \"deptno\" from \"emps\" where \"deptno\" < 10 group by \"deptno\"",
+      JdbcTest.HR_MODEL);
+  }
+
+  @Test public void testAggregateMaterializationNoAggregateFuncs8() {
+    checkNoMaterialize(
+      "select \"empid\" from \"emps\" group by \"empid\", \"deptno\"",
+      "select \"deptno\" from \"emps\" group by \"deptno\"",
+      JdbcTest.HR_MODEL);
+  }
+
+  @Test public void testAggregateMaterializationNoAggregateFuncs9() {
+    checkNoMaterialize(
+      "select \"empid\", \"deptno\" from \"emps\"\n"
+          + "where \"salary\" > 1000 group by \"name\", \"empid\", \"deptno\"",
+      "select \"empid\" from \"emps\"\n"
+          + "where \"salary\" > 2000 group by \"name\", \"empid\"",
+      JdbcTest.HR_MODEL);
+  }
+
+  @Test public void testAggregateMaterializationAggregateFuncs1() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" group by \"empid\", \"deptno\"",
+      "select \"deptno\" from \"emps\" group by \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationAggregateFuncs2() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" group by \"empid\", \"deptno\"",
+      "select \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" group by \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}], C=[$SUM0($2)], S=[$SUM0($3)])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationAggregateFuncs3() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" group by \"empid\", \"deptno\"",
+      "select \"deptno\", \"empid\", sum(\"empid\") as s, count(*) as c\n"
+          + "from \"emps\" group by \"empid\", \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t1], empid=[$t0], "
+              + "S=[$t3], C=[$t2])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationAggregateFuncs4() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
+      "select \"deptno\", sum(\"empid\") as s\n"
+          + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}], S=[$SUM0($3)])\n"
+              + "  EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[>($t1, $t4)], "
+              + "proj#0..3=[{exprs}], $condition=[$t5])\n"
+              + "    EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationAggregateFuncs5() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
+          + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
+      "select \"deptno\", sum(\"empid\") + 1 as s\n"
+          + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], "
+              + "deptno=[$t0], S=[$t3])\n"
+              + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($3)])\n"
+              + "    EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[>($t1, $t4)], "
+              + "proj#0..3=[{exprs}], $condition=[$t5])\n"
+              + "      EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testAggregateMaterializationAggregateFuncs6() {
+    checkNoMaterialize(
+      "select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") + 2 as s\n"
+          + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
+      "select \"deptno\", sum(\"empid\") + 1 as s\n"
+          + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
+      JdbcTest.HR_MODEL);
+  }
+
+  @Test public void testAggregateMaterializationAggregateFuncs7() {
+    checkMaterialize(
+      "select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
+          + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
+      "select \"deptno\" + 1, sum(\"empid\") + 1 as s\n"
+          + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t0, $t2)], "
+              + "expr#4=[+($t1, $t2)], EXPR$0=[$t3], S=[$t4])\n"
+              + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($3)])\n"
+              + "    EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[>($t1, $t4)], "
+              + "proj#0..3=[{exprs}], $condition=[$t5])\n"
+              + "      EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Ignore
+  @Test public void testAggregateMaterializationAggregateFuncs8() {
+    // TODO: It should work, but top project in the query is not matched by the planner.
+    // It needs further checking.
+    checkMaterialize(
+      "select \"empid\", \"deptno\" + 1, count(*) + 1 as c, sum(\"empid\") as s\n"
+          + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
+      "select \"deptno\" + 1, sum(\"empid\") + 1 as s\n"
+          + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"");
+  }
+
+  @Test public void testJoinAggregateMaterializationNoAggregateFuncs1() {
+    checkMaterialize(
+      "select \"empid\", \"depts\".\"deptno\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 10\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      "select \"empid\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[>($t1, $t2)], "
+              + "empid=[$t0], $condition=[$t3])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinAggregateMaterializationNoAggregateFuncs2() {
+    checkMaterialize(
+      "select \"depts\".\"deptno\", \"empid\" from \"depts\"\n"
+          + "join \"emps\" using (\"deptno\") where \"depts\".\"deptno\" > 10\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      "select \"empid\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[>($t0, $t2)], "
+              + "empid=[$t1], $condition=[$t3])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinAggregateMaterializationNoAggregateFuncs3() {
+    // It does not match, Project on top of query
+    checkNoMaterialize(
+      "select \"empid\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 10\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      "select \"empid\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      JdbcTest.HR_MODEL);
+  }
+
+  @Test public void testJoinAggregateMaterializationNoAggregateFuncs4() {
+    checkMaterialize(
+      "select \"empid\", \"depts\".\"deptno\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"emps\".\"deptno\" > 10\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      "select \"empid\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[>($t1, $t2)], "
+              + "empid=[$t0], $condition=[$t3])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinAggregateMaterializationNoAggregateFuncs5() {
+    checkMaterialize(
+      "select \"depts\".\"deptno\", \"emps\".\"empid\" from \"depts\"\n"
+          + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 10\n"
+          + "group by \"depts\".\"deptno\", \"emps\".\"empid\"",
+      "select \"depts\".\"deptno\" from \"depts\"\n"
+          + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 15\n"
+          + "group by \"depts\".\"deptno\", \"emps\".\"empid\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[15], expr#3=[>($t1, $t2)], "
+              + "deptno=[$t0], $condition=[$t3])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinAggregateMaterializationNoAggregateFuncs6() {
+    checkMaterialize(
+      "select \"depts\".\"deptno\", \"emps\".\"empid\" from \"depts\"\n"
+          + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 10\n"
+          + "group by \"depts\".\"deptno\", \"emps\".\"empid\"",
+      "select \"depts\".\"deptno\" from \"depts\"\n"
+          + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 15\n"
+          + "group by \"depts\".\"deptno\"",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{0}])\n"
+              + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[15], expr#3=[>($t1, $t2)], "
+              + "proj#0..1=[{exprs}], $condition=[$t3])\n"
+              + "    EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinMaterialization4() {
+    checkMaterialize(
+      "select \"empid\" \"deptno\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\")",
+      "select \"empid\" \"deptno\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\") where \"empid\" = 1",
+      JdbcTest.HR_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):INTEGER NOT NULL], expr#2=[1], "
+              + "expr#3=[=($t1, $t2)], deptno=[$t0], $condition=[$t3])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
   @Test public void testSubQuery() {
     String q = "select \"empid\", \"deptno\", \"salary\" from \"emps\" e1\n"
         + "where \"empid\" = (\n"


[06/11] calcite git commit: [CALCITE-1731] Materialized view rewriting for join and aggregate operators

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/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 29603b1..bce7626 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.adapter.java.ReflectiveSchema;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.materialize.MaterializationService;
 import org.apache.calcite.plan.RelOptPlanner;
@@ -24,6 +25,8 @@ import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.SubstitutionVisitor;
 import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
+import org.apache.calcite.rel.RelReferentialConstraintImpl;
 import org.apache.calcite.rel.RelVisitor;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.rules.MaterializedViewJoinRule;
@@ -36,11 +39,19 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexSimplify;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.test.JdbcTest.Department;
+import org.apache.calcite.test.JdbcTest.Dependent;
+import org.apache.calcite.test.JdbcTest.Employee;
+import org.apache.calcite.test.JdbcTest.Location;
 import org.apache.calcite.tools.RuleSet;
 import org.apache.calcite.tools.RuleSets;
 import org.apache.calcite.util.JsonBuilder;
+import org.apache.calcite.util.Smalls;
 import org.apache.calcite.util.TryThreadLocal;
+import org.apache.calcite.util.mapping.IntPair;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
@@ -52,6 +63,7 @@ import org.junit.Test;
 import java.math.BigDecimal;
 import java.sql.ResultSet;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -86,6 +98,25 @@ public class MaterializationTest {
   CASE_INSENSITIVE_LIST_LIST_COMPARATOR =
       CASE_INSENSITIVE_LIST_COMPARATOR.lexicographical();
 
+  private static final String HR_FKUK_SCHEMA = "{\n"
+      + "       type: 'custom',\n"
+      + "       name: 'hr',\n"
+      + "       factory: '"
+      + ReflectiveSchema.Factory.class.getName()
+      + "',\n"
+      + "       operand: {\n"
+      + "         class: '" + HrFKUKSchema.class.getName() + "'\n"
+      + "       }\n"
+      + "     }\n";
+
+  private static final String HR_FKUK_MODEL = "{\n"
+      + "  version: '1.0',\n"
+      + "  defaultSchema: 'hr',\n"
+      + "   schemas: [\n"
+      + HR_FKUK_SCHEMA
+      + "   ]\n"
+      + "}";
+
   final JavaTypeFactoryImpl typeFactory =
       new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
   private final RexBuilder rexBuilder = new RexBuilder(typeFactory);
@@ -122,7 +153,7 @@ public class MaterializationTest {
   @Test public void testFilter() {
     CalciteAssert.that()
         .withMaterializations(
-            JdbcTest.HR_MODEL,
+            HR_FKUK_MODEL,
             "m0",
             "select * from \"emps\" where \"deptno\" = 10")
         .query(
@@ -137,7 +168,7 @@ public class MaterializationTest {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(
-              JdbcTest.HR_MODEL,
+              HR_FKUK_MODEL,
               "m0",
               "select \"deptno\", \"empid\" from \"emps\"")
           .query(
@@ -151,14 +182,14 @@ public class MaterializationTest {
   /** Checks that a given query can use a materialized view with a given
    * definition. */
   private void checkMaterialize(String materialize, String query) {
-    checkMaterialize(materialize, query, JdbcTest.HR_MODEL, CONTAINS_M0,
+    checkMaterialize(materialize, query, HR_FKUK_MODEL, CONTAINS_M0,
         RuleSets.ofList(ImmutableList.<RelOptRule>of()));
   }
 
   /** Checks that a given query can use a materialized view with a given
    * definition. */
   private void checkMaterializeWithRules(String materialize, String query, RuleSet rules) {
-    checkMaterialize(materialize, query, JdbcTest.HR_MODEL, CONTAINS_M0, rules);
+    checkMaterialize(materialize, query, HR_FKUK_MODEL, CONTAINS_M0, rules);
   }
 
   /** Checks that a given query can use a materialized view with a given
@@ -247,7 +278,7 @@ public class MaterializationTest {
     checkNoMaterialize(
         "select \"deptno\" - 10 as \"x\", \"empid\" + 1, \"name\" from \"emps\"",
         "select \"name\" from \"emps\" where \"deptno\" + 10 = 20",
-        JdbcTest.HR_MODEL);
+        HR_FKUK_MODEL);
   }
 
   /** As {@link #testFilterQueryOnProjectView3()} but also contains an
@@ -258,10 +289,10 @@ public class MaterializationTest {
         + "from \"emps\"",
         "select \"name\", \"empid\" + 1 as e\n"
         + "from \"emps\" where \"deptno\" - 10 = 2",
-        JdbcTest.HR_MODEL,
+        HR_FKUK_MODEL,
         CalciteAssert.checkResultContains(
             "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[2], "
-                + "expr#4=[=($t0, $t3)], name=[$t2], E=[$t1], $condition=[$t4])\n"
+                + "expr#4=[=($t0, $t3)], name=[$t2], EE=[$t1], $condition=[$t4])\n"
                 + "  EnumerableTableScan(table=[[hr, m0]]"));
   }
 
@@ -270,7 +301,7 @@ public class MaterializationTest {
     checkNoMaterialize(
         "select \"deptno\" - 10 as \"x\", \"empid\"  from \"emps\"",
         "select \"name\" from \"emps\" where \"deptno\" - 10 = 0",
-        JdbcTest.HR_MODEL);
+        HR_FKUK_MODEL);
   }
 
   /** As {@link #testFilterQueryOnProjectView3()} but also contains an
@@ -279,7 +310,7 @@ public class MaterializationTest {
     checkNoMaterialize(
         "select \"deptno\" - 10 as \"x\", \"empid\" + 1, \"name\" from \"emps\"",
         "select \"name\", \"empid\" + 2 from \"emps\" where \"deptno\" - 10 = 0",
-        JdbcTest.HR_MODEL);
+        HR_FKUK_MODEL);
   }
 
   /** Test case for
@@ -355,7 +386,7 @@ public class MaterializationTest {
         "select \"deptno\", \"empid\", \"name\" from \"emps\" "
             + "where \"deptno\" = 10 or \"deptno\" = 20 or \"empid\" < 160",
         "select \"empid\" + 1 as x, \"name\" from \"emps\" where \"deptno\" = 10",
-        JdbcTest.HR_MODEL,
+        HR_FKUK_MODEL,
         CalciteAssert.checkResultContains(
             "EnumerableCalcRel(expr#0..2=[{inputs}], expr#3=[1], "
                 + "expr#4=[+($t1, $t3)], X=[$t4], name=[$t2], condition=?)\n"
@@ -407,7 +438,7 @@ public class MaterializationTest {
     checkNoMaterialize(
             "select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10",
             "select \"name\", \"empid\" from \"emps\" where \"deptno\" > 30",
-            JdbcTest.HR_MODEL);
+            HR_FKUK_MODEL);
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is weaker in
@@ -417,7 +448,7 @@ public class MaterializationTest {
             "select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10",
             "select \"name\", \"empid\" from \"emps\" "
                 + "where \"deptno\" > 30 or \"empid\" > 10",
-            JdbcTest.HR_MODEL);
+            HR_FKUK_MODEL);
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition currently
@@ -428,7 +459,7 @@ public class MaterializationTest {
                     + "and \"name\" = \'calcite\'",
             "select \"name\", \"empid\" from \"emps\" where \"deptno\" > 30 "
                     + "or \"empid\" > 10",
-            JdbcTest.HR_MODEL);
+            HR_FKUK_MODEL);
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is weaker in
@@ -440,7 +471,7 @@ public class MaterializationTest {
                     + "(\"salary\" < 1111.9 and \"deptno\" > 10)"
                     + "or (\"empid\" > 400 and \"salary\" > 5000)",
             "select \"name\" from \"emps\" where \"deptno\" > 30 and \"salary\" > 3000",
-            JdbcTest.HR_MODEL);
+            HR_FKUK_MODEL);
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition of
@@ -450,7 +481,7 @@ public class MaterializationTest {
     checkNoMaterialize(
             "select \"name\", \"deptno\" from \"emps\" where \"salary\" > 2000.5",
             "select \"name\" from \"emps\" where \"deptno\" > 30 and \"salary\" > 3000",
-            JdbcTest.HR_MODEL);
+            HR_FKUK_MODEL);
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is weaker in
@@ -463,7 +494,7 @@ public class MaterializationTest {
                     + "or (\"empid\" > 400 and \"salary\" > 5000)",
             "select \"name\" from \"emps\" where \"salary\" > 1000 "
                     + "or (\"deptno\" > 30 and \"salary\" > 3000)",
-            JdbcTest.HR_MODEL);
+            HR_FKUK_MODEL);
   }
 
   /** As {@link #testFilterQueryOnFilterView7()} but columns in materialized
@@ -506,7 +537,7 @@ public class MaterializationTest {
         "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s from \"emps\" "
             + "group by \"empid\", \"deptno\"",
         "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\"",
-        JdbcTest.HR_MODEL,
+        HR_FKUK_MODEL,
         CalciteAssert.checkResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], "
                 + "expr#3=[+($t1, $t2)], C=[$t3], deptno=[$t0])\n"
@@ -522,7 +553,7 @@ public class MaterializationTest {
     checkMaterialize(
         "select \"deptno\", count(*) as c, \"empid\" + 2, sum(\"empid\") as s from \"emps\" group by \"empid\", \"deptno\"",
         "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\"",
-        JdbcTest.HR_MODEL,
+        HR_FKUK_MODEL,
         CalciteAssert.checkResultContains(
             "xxx"));
   }
@@ -928,7 +959,7 @@ public class MaterializationTest {
     String q = "select *\n"
         + "from (select * from \"emps\" union all select * from \"emps\")\n"
         + "join \"depts\" using (\"deptno\")";
-    checkNoMaterialize(q, q, JdbcTest.HR_MODEL);
+    checkNoMaterialize(q, q, HR_FKUK_MODEL);
   }
 
   @Test public void testJoinMaterialization() {
@@ -965,7 +996,7 @@ public class MaterializationTest {
     String q = "select * from \"emps\" where \"empid\" > 300\n"
         + "union all select * from \"emps\" where \"empid\" < 200";
     String m = "select * from \"emps\" where \"empid\" < 500";
-    checkMaterialize(m, q, JdbcTest.HR_MODEL,
+    checkMaterialize(m, q, HR_FKUK_MODEL,
         CalciteAssert.checkResultContains(
             "EnumerableTableScan(table=[[hr, m0]])", 1));
   }
@@ -974,7 +1005,7 @@ public class MaterializationTest {
     checkMaterialize(
       "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
       "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableTableScan(table=[[hr, m0]])"));
   }
@@ -983,7 +1014,7 @@ public class MaterializationTest {
     checkMaterialize(
       "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
       "select \"deptno\" from \"emps\" group by \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableAggregate(group=[{1}])\n"
               + "  EnumerableTableScan(table=[[hr, m0]])"));
@@ -993,14 +1024,14 @@ public class MaterializationTest {
     checkNoMaterialize(
       "select \"deptno\" from \"emps\" group by \"deptno\"",
       "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
-      JdbcTest.HR_MODEL);
+      HR_FKUK_MODEL);
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs4() {
     checkMaterialize(
       "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"empid\", \"deptno\"",
       "select \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableAggregate(group=[{1}])\n"
               + "  EnumerableTableScan(table=[[hr, m0]])"));
@@ -1010,14 +1041,14 @@ public class MaterializationTest {
     checkNoMaterialize(
       "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" = 5 group by \"empid\", \"deptno\"",
       "select \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"deptno\"",
-      JdbcTest.HR_MODEL);
+      HR_FKUK_MODEL);
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs6() {
     checkMaterialize(
       "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" > 5 group by \"empid\", \"deptno\"",
       "select \"deptno\" from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableAggregate(group=[{1}])\n"
               + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[10], expr#3=[>($t1, $t2)], "
@@ -1029,14 +1060,14 @@ public class MaterializationTest {
     checkNoMaterialize(
       "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" > 5 group by \"empid\", \"deptno\"",
       "select \"deptno\" from \"emps\" where \"deptno\" < 10 group by \"deptno\"",
-      JdbcTest.HR_MODEL);
+      HR_FKUK_MODEL);
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs8() {
     checkNoMaterialize(
       "select \"empid\" from \"emps\" group by \"empid\", \"deptno\"",
       "select \"deptno\" from \"emps\" group by \"deptno\"",
-      JdbcTest.HR_MODEL);
+      HR_FKUK_MODEL);
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs9() {
@@ -1045,7 +1076,7 @@ public class MaterializationTest {
           + "where \"salary\" > 1000 group by \"name\", \"empid\", \"deptno\"",
       "select \"empid\" from \"emps\"\n"
           + "where \"salary\" > 2000 group by \"name\", \"empid\"",
-      JdbcTest.HR_MODEL);
+      HR_FKUK_MODEL);
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs1() {
@@ -1053,7 +1084,7 @@ public class MaterializationTest {
       "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
           + "from \"emps\" group by \"empid\", \"deptno\"",
       "select \"deptno\" from \"emps\" group by \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableAggregate(group=[{1}])\n"
               + "  EnumerableTableScan(table=[[hr, m0]])"));
@@ -1065,7 +1096,7 @@ public class MaterializationTest {
           + "from \"emps\" group by \"empid\", \"deptno\"",
       "select \"deptno\", count(*) as c, sum(\"empid\") as s\n"
           + "from \"emps\" group by \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableAggregate(group=[{1}], C=[$SUM0($2)], S=[$SUM0($3)])\n"
               + "  EnumerableTableScan(table=[[hr, m0]])"));
@@ -1077,7 +1108,7 @@ public class MaterializationTest {
           + "from \"emps\" group by \"empid\", \"deptno\"",
       "select \"deptno\", \"empid\", sum(\"empid\") as s, count(*) as c\n"
           + "from \"emps\" group by \"empid\", \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t1], empid=[$t0], "
               + "S=[$t3], C=[$t2])\n"
@@ -1090,7 +1121,7 @@ public class MaterializationTest {
           + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
       "select \"deptno\", sum(\"empid\") as s\n"
           + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableAggregate(group=[{1}], S=[$SUM0($3)])\n"
               + "  EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[>($t1, $t4)], "
@@ -1104,7 +1135,7 @@ public class MaterializationTest {
           + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
       "select \"deptno\", sum(\"empid\") + 1 as s\n"
           + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], "
               + "deptno=[$t0], S=[$t3])\n"
@@ -1120,7 +1151,7 @@ public class MaterializationTest {
           + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
       "select \"deptno\", sum(\"empid\") + 1 as s\n"
           + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-      JdbcTest.HR_MODEL);
+      HR_FKUK_MODEL);
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs7() {
@@ -1129,7 +1160,7 @@ public class MaterializationTest {
           + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
       "select \"deptno\" + 1, sum(\"empid\") + 1 as s\n"
           + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t0, $t2)], "
               + "expr#4=[+($t1, $t2)], EXPR$0=[$t3], S=[$t4])\n"
@@ -1158,7 +1189,7 @@ public class MaterializationTest {
       "select \"empid\" from \"emps\"\n"
           + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
           + "group by \"empid\", \"depts\".\"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[>($t1, $t2)], "
               + "empid=[$t0], $condition=[$t3])\n"
@@ -1173,7 +1204,7 @@ public class MaterializationTest {
       "select \"empid\" from \"emps\"\n"
           + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
           + "group by \"empid\", \"depts\".\"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[>($t0, $t2)], "
               + "empid=[$t1], $condition=[$t3])\n"
@@ -1189,7 +1220,7 @@ public class MaterializationTest {
       "select \"empid\" from \"emps\"\n"
           + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
           + "group by \"empid\", \"depts\".\"deptno\"",
-      JdbcTest.HR_MODEL);
+      HR_FKUK_MODEL);
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs4() {
@@ -1200,7 +1231,7 @@ public class MaterializationTest {
       "select \"empid\" from \"emps\"\n"
           + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
           + "group by \"empid\", \"depts\".\"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[>($t1, $t2)], "
               + "empid=[$t0], $condition=[$t3])\n"
@@ -1215,7 +1246,7 @@ public class MaterializationTest {
       "select \"depts\".\"deptno\" from \"depts\"\n"
           + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 15\n"
           + "group by \"depts\".\"deptno\", \"emps\".\"empid\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[15], expr#3=[>($t1, $t2)], "
               + "deptno=[$t0], $condition=[$t3])\n"
@@ -1230,7 +1261,7 @@ public class MaterializationTest {
       "select \"depts\".\"deptno\" from \"depts\"\n"
           + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 15\n"
           + "group by \"depts\".\"deptno\"",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableAggregate(group=[{0}])\n"
               + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[15], expr#3=[>($t1, $t2)], "
@@ -1238,26 +1269,169 @@ public class MaterializationTest {
               + "    EnumerableTableScan(table=[[hr, m0]])"));
   }
 
+  @Test public void testJoinAggregateMaterializationAggregateFuncs1() {
+    // This test relies on FK-UK relationship
+    checkMaterialize(
+      "select \"empid\", \"depts\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      "select \"deptno\" from \"emps\" group by \"deptno\"",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinAggregateMaterializationAggregateFuncs2() {
+    checkMaterialize(
+      "select \"empid\", \"emps\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+          + "group by \"empid\", \"emps\".\"deptno\"",
+      "select \"depts\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+          + "group by \"depts\".\"deptno\"",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}], C=[$SUM0($2)], S=[$SUM0($3)])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinAggregateMaterializationAggregateFuncs3() {
+    // This test relies on FK-UK relationship
+    checkMaterialize(
+      "select \"empid\", \"depts\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+          + "group by \"empid\", \"depts\".\"deptno\"",
+      "select \"deptno\", \"empid\", sum(\"empid\") as s, count(*) as c\n"
+          + "from \"emps\" group by \"empid\", \"deptno\"",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t1], empid=[$t0], "
+              + "S=[$t3], C=[$t2])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinAggregateMaterializationAggregateFuncs4() {
+    checkMaterialize(
+      "select \"empid\", \"emps\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
+          + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+          + "where \"emps\".\"deptno\" >= 10 group by \"empid\", \"emps\".\"deptno\"",
+      "select \"depts\".\"deptno\", sum(\"empid\") as s\n"
+          + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+          + "where \"emps\".\"deptno\" > 10 group by \"depts\".\"deptno\"",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableAggregate(group=[{1}], S=[$SUM0($3)])\n"
+              + "  EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[>($t1, $t4)], "
+              + "proj#0..3=[{exprs}], $condition=[$t5])\n"
+              + "    EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinAggregateMaterializationAggregateFuncs5() {
+    checkMaterialize(
+      "select \"empid\", \"depts\".\"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
+          + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+          + "where \"depts\".\"deptno\" >= 10 group by \"empid\", \"depts\".\"deptno\"",
+      "select \"depts\".\"deptno\", sum(\"empid\") + 1 as s\n"
+          + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+          + "where \"depts\".\"deptno\" > 10 group by \"depts\".\"deptno\"",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], "
+              + "deptno=[$t0], S=[$t3])\n"
+              + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($3)])\n"
+              + "    EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[>($t1, $t4)], "
+              + "proj#0..3=[{exprs}], $condition=[$t5])\n"
+              + "      EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
   @Test public void testJoinMaterialization4() {
     checkMaterialize(
       "select \"empid\" \"deptno\" from \"emps\"\n"
           + "join \"depts\" using (\"deptno\")",
       "select \"empid\" \"deptno\" from \"emps\"\n"
           + "join \"depts\" using (\"deptno\") where \"empid\" = 1",
-      JdbcTest.HR_MODEL,
+      HR_FKUK_MODEL,
       CalciteAssert.checkResultContains(
           "EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):INTEGER NOT NULL], expr#2=[1], "
               + "expr#3=[=($t1, $t2)], deptno=[$t0], $condition=[$t3])\n"
               + "  EnumerableTableScan(table=[[hr, m0]])"));
   }
 
+  @Test public void testJoinMaterializationUKFK1() {
+    checkMaterialize(
+      "select \"a\".\"empid\" \"deptno\" from\n"
+          + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
+          + "join \"depts\" using (\"deptno\")\n"
+          + "join \"dependents\" using (\"empid\")",
+      "select \"a\".\"empid\" from \n"
+          + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
+          + "join \"dependents\" using (\"empid\")\n",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "PLAN=EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinMaterializationUKFK2() {
+    checkMaterialize(
+      "select \"a\".\"empid\", \"a\".\"deptno\" from\n"
+          + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
+          + "join \"depts\" using (\"deptno\")\n"
+          + "join \"dependents\" using (\"empid\")",
+      "select \"a\".\"empid\" from \n"
+          + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
+          + "join \"dependents\" using (\"empid\")\n",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], empid=[$t0])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinMaterializationUKFK3() {
+    checkNoMaterialize(
+      "select \"a\".\"empid\", \"a\".\"deptno\" from\n"
+          + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
+          + "join \"depts\" using (\"deptno\")\n"
+          + "join \"dependents\" using (\"empid\")",
+      "select \"a\".\"name\" from \n"
+          + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
+          + "join \"dependents\" using (\"empid\")\n",
+      HR_FKUK_MODEL);
+  }
+
+  @Test public void testJoinMaterializationUKFK4() {
+    checkMaterialize(
+      "select \"empid\" \"deptno\" from\n"
+          + "(select * from \"emps\" where \"empid\" = 1)\n"
+          + "join \"depts\" using (\"deptno\")",
+      "select \"empid\" from \"emps\" where \"empid\" = 1\n",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "PLAN=EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
+  @Test public void testJoinMaterializationUKFK5() {
+    checkMaterialize(
+      "select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
+          + "join \"depts\" using (\"deptno\")\n"
+          + "join \"dependents\" using (\"empid\")"
+          + "where \"emps\".\"empid\" = 1",
+      "select \"emps\".\"empid\" from \"emps\"\n"
+          + "join \"dependents\" using (\"empid\")\n"
+          + "where \"emps\".\"empid\" = 1",
+      HR_FKUK_MODEL,
+      CalciteAssert.checkResultContains(
+          "EnumerableCalc(expr#0..1=[{inputs}], empid=[$t0])\n"
+              + "  EnumerableTableScan(table=[[hr, m0]])"));
+  }
+
   @Test public void testSubQuery() {
     String q = "select \"empid\", \"deptno\", \"salary\" from \"emps\" e1\n"
         + "where \"empid\" = (\n"
         + "  select max(\"empid\") from \"emps\"\n"
         + "  where \"deptno\" = e1.\"deptno\")";
     final String m = "select \"empid\", \"deptno\" from \"emps\"\n";
-    checkMaterialize(m, q, JdbcTest.HR_MODEL,
+    checkMaterialize(m, q, HR_FKUK_MODEL,
         CalciteAssert.checkResultContains(
             "EnumerableTableScan(table=[[hr, m0]])", 1));
   }
@@ -1273,7 +1447,7 @@ public class MaterializationTest {
     try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
-          .withMaterializations(JdbcTest.HR_MODEL,
+          .withMaterializations(HR_FKUK_MODEL,
               "m0", m)
           .query(q)
           .withHook(Hook.SUB,
@@ -1300,7 +1474,7 @@ public class MaterializationTest {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(
-              JdbcTest.HR_MODEL,
+              HR_FKUK_MODEL,
               new Function<JsonBuilder, List<Object>>() {
                 public List<Object> apply(JsonBuilder builder) {
                   final Map<String, Object> map = builder.map();
@@ -1364,7 +1538,7 @@ public class MaterializationTest {
         + "from (select * from \"emps\" where \"empid\" < 300)\n"
         + "join (select * from \"emps\" where \"empid\" < 200) using (\"empid\")";
     String m = "select * from \"emps\" where \"empid\" < 500";
-    checkMaterialize(m, q, JdbcTest.HR_MODEL,
+    checkMaterialize(m, q, HR_FKUK_MODEL,
         CalciteAssert.checkResultContains(
             "EnumerableTableScan(table=[[hr, m0]])", 2));
   }
@@ -1376,7 +1550,7 @@ public class MaterializationTest {
     try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
-          .withMaterializations(JdbcTest.HR_MODEL,
+          .withMaterializations(HR_FKUK_MODEL,
               "m0", "select \"deptno\", count(*) as c, sum(\"empid\") as s from \"emps\" group by \"deptno\"",
               "m1", "select * from \"emps\" where \"empid\" < 500")
           .query(q)
@@ -1394,7 +1568,7 @@ public class MaterializationTest {
     try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
-          .withMaterializations(JdbcTest.HR_MODEL,
+          .withMaterializations(HR_FKUK_MODEL,
               "m0", "select * from \"emps\" where \"empid\" < 500")
           .query(q)
           .enableMaterializations(true)
@@ -1412,7 +1586,7 @@ public class MaterializationTest {
     try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
-          .withMaterializations(JdbcTest.HR_MODEL,
+          .withMaterializations(HR_FKUK_MODEL,
               "m0", "select * from \"emps\" where \"empid\" < 500",
               "m1", "select * from \"depts\" where \"deptno\" > 100")
           .query(q)
@@ -1442,7 +1616,7 @@ public class MaterializationTest {
       MaterializationService.setThreadLocal();
       final List<List<List<String>>> substitutedNames = new ArrayList<>();
       CalciteAssert.that()
-          .withMaterializations(JdbcTest.HR_MODEL,
+          .withMaterializations(HR_FKUK_MODEL,
               "m0", "select * from \"emps\" where \"empid\" < 300",
               "m1", "select * from \"emps\" where \"empid\" < 600")
           .query(q)
@@ -1486,7 +1660,7 @@ public class MaterializationTest {
       MaterializationService.setThreadLocal();
       final List<List<List<String>>> substitutedNames = new ArrayList<>();
       CalciteAssert.that()
-          .withMaterializations(JdbcTest.HR_MODEL,
+          .withMaterializations(HR_FKUK_MODEL,
               "m0", "select * from \"emps\" where \"empid\" < 300",
               "m1", "select * from \"emps\" where \"empid\" < 600",
               "m2", "select * from \"m1\"")
@@ -1542,6 +1716,50 @@ public class MaterializationTest {
       super.visit(node, ordinal, parent);
     }
   }
+
+  /**
+   * Hr schema with FK-UK relationship.
+   */
+  public static class HrFKUKSchema {
+    @Override public String toString() {
+      return "HrFKUKSchema";
+    }
+
+    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], emps[3]),
+          new Location(-122, 38)),
+      new Department(30, "Marketing", Collections.<Employee>emptyList(),
+          new Location(0, 52)),
+      new Department(20, "HR", Collections.singletonList(emps[1]), null),
+    };
+    public final Dependent[] dependents = {
+      new Dependent(10, "Michael"),
+      new Dependent(10, "Jane"),
+    };
+    public final Dependent[] locations = {
+      new Dependent(10, "San Francisco"),
+      new Dependent(20, "San Diego"),
+    };
+
+    public final RelReferentialConstraint rcs0 =
+        RelReferentialConstraintImpl.of(
+            ImmutableList.of("hr", "emps"), ImmutableList.of("hr", "depts"),
+            ImmutableList.of(IntPair.of(1, 0)));
+
+    public QueryableTable foo(int count) {
+      return Smalls.generateStrings(count);
+    }
+
+    public TranslatableTable view(String s) {
+      return Smalls.view(s);
+    }
+  }
 }
 
 // End MaterializationTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
index 52733c4..c3e112c 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalTableScan;
@@ -669,6 +670,8 @@ public class MockCatalogReader extends CalciteCatalogReader {
     protected final List<Map.Entry<String, RelDataType>> columnList =
         new ArrayList<>();
     protected final List<Integer> keyList = new ArrayList<>();
+    protected final List<RelReferentialConstraint> referentialConstraints =
+        new ArrayList<>();
     protected RelDataType rowType;
     protected List<RelCollation> collationList;
     protected final List<String> names;
@@ -859,6 +862,10 @@ public class MockCatalogReader extends CalciteCatalogReader {
           && columns.contains(ImmutableBitSet.of(keyList));
     }
 
+    public List<RelReferentialConstraint> getReferentialConstraints() {
+      return referentialConstraints;
+    }
+
     public RelDataType getRowType() {
       return rowType;
     }
@@ -1461,6 +1468,10 @@ public class MockCatalogReader extends CalciteCatalogReader {
           return table.isKey(columns);
         }
 
+        public List<RelReferentialConstraint> getReferentialConstraints() {
+          return table.getReferentialConstraints();
+        }
+
         public List<RelCollation> getCollations() {
           return table.collationList;
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/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 536bc4e..0e0d234 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalTableScan;
@@ -398,6 +399,10 @@ public abstract class SqlToRelTestBase {
         return false;
       }
 
+      public List<RelReferentialConstraint> getReferentialConstraints() {
+        return ImmutableList.of();
+      }
+
       public Expression getExpression(Class clazz) {
         return null;
       }
@@ -467,6 +472,10 @@ public abstract class SqlToRelTestBase {
     public boolean isKey(ImmutableBitSet columns) {
       return parent.isKey(columns);
     }
+
+    public List<RelReferentialConstraint> getReferentialConstraints() {
+      return parent.getReferentialConstraints();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/1f81e135/site/_docs/materialized_views.md
----------------------------------------------------------------------
diff --git a/site/_docs/materialized_views.md b/site/_docs/materialized_views.md
index 9856a87..cae361b 100644
--- a/site/_docs/materialized_views.md
+++ b/site/_docs/materialized_views.md
@@ -82,6 +82,5 @@ shortcomings that we plan to address with follow-up extensions:
 * It does not produce rewritings using Union operators, e.g., a given query could be partially answered from the
 {mv} (year = 2014) and from the query (not(year=2014)). This can be useful if {mv} is stored in a system such as
 Druid.
-* Currently query and {mv} must use the same tables.
 
 This rule is currently enabled by default.


[09/11] calcite git commit: [CALCITE-1731] Materialized view rewriting for join and aggregate operators

Posted by jc...@apache.org.
[CALCITE-1731] Materialized view rewriting for join and aggregate operators

* Remove previous join rewriting rule (MaterializedViewJoinRule.java) and update documentation.


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

Branch: refs/heads/master
Commit: 27ca3109539eba4324b3bf4fcf4065cb7a5d300c
Parents: 1f81e13
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Apr 25 13:58:40 2017 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Apr 26 20:03:11 2017 +0100

----------------------------------------------------------------------
 .../rel/rules/MaterializedViewJoinRule.java     | 373 -------------------
 .../calcite/test/MaterializationTest.java       |   5 +-
 site/_docs/materialized_views.md                |  25 +-
 3 files changed, 9 insertions(+), 394 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/27ca3109/core/src/main/java/org/apache/calcite/rel/rules/MaterializedViewJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MaterializedViewJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/MaterializedViewJoinRule.java
deleted file mode 100644
index f6fb009..0000000
--- a/core/src/main/java/org/apache/calcite/rel/rules/MaterializedViewJoinRule.java
+++ /dev/null
@@ -1,373 +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.RelOptMaterialization;
-import org.apache.calcite.plan.RelOptMaterializations;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.hep.HepPlanner;
-import org.apache.calcite.plan.hep.HepProgram;
-import org.apache.calcite.plan.hep.HepProgramBuilder;
-import org.apache.calcite.plan.volcano.VolcanoPlanner;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.type.SqlTypeUtil;
-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 com.google.common.collect.Lists;
-
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-/**
- * Planner rule that converts joins of multiple tables into a matching
- * materialized view
- */
-public class MaterializedViewJoinRule extends RelOptRule {
-  public static final MaterializedViewJoinRule INSTANCE_PROJECT =
-      new MaterializedViewJoinRule(
-          operand(LogicalProject.class,
-              operand(Join.class,
-                  operand(Project.class,
-                      operand(TableScan.class, none())),
-                  operand(Project.class,
-                      operand(TableScan.class, none())))),
-          RelFactories.LOGICAL_BUILDER,
-          "MaterializedViewJoinRule(Project-Project)");
-
-  public static final MaterializedViewJoinRule INSTANCE_TABLE_SCAN =
-      new MaterializedViewJoinRule(
-          operand(LogicalProject.class,
-              operand(Join.class,
-                  operand(TableScan.class, none()),
-                  operand(TableScan.class, none()))),
-          RelFactories.LOGICAL_BUILDER,
-          "MaterializedViewJoinRule(TableScan-TableScan)");
-
-  private final HepProgram multiJoinProgram = new HepProgramBuilder()
-      .addRuleInstance(ProjectRemoveRule.INSTANCE)
-      .addRuleInstance(ProjectJoinTransposeRule.INSTANCE)
-      .addRuleInstance(JoinToMultiJoinRule.INSTANCE)
-      .addRuleInstance(ProjectMultiJoinMergeRule.INSTANCE)
-      .addRuleInstance(FilterMultiJoinMergeRule.INSTANCE)
-      .build();
-
-  //~ Constructors -----------------------------------------------------------
-
-  /** Creates a MaterializedViewJoinRule. */
-  protected MaterializedViewJoinRule(RelOptRuleOperand operand, RelBuilderFactory relBuilderFactory,
-        String description) {
-    super(operand, relBuilderFactory, description);
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  public void onMatch(RelOptRuleCall call) {
-    final Project originalProject = call.rel(0);
-    // Rebuild the tree
-    final RelNode leftInput;
-    final RelNode rightInput;
-    if (call.getRelList().size() == 6) {
-      leftInput = call.rel(2).copy(call.rel(2).getTraitSet(), ImmutableList.of(call.rel(3)));
-      rightInput = call.rel(4).copy(call.rel(4).getTraitSet(), ImmutableList.of(call.rel(5)));
-    } else {
-      leftInput = call.rel(2);
-      rightInput = call.rel(3);
-    }
-    final RelNode join = call.rel(1).copy(call.rel(1).getTraitSet(),
-        ImmutableList.of(leftInput, rightInput));
-    final RelNode project = call.rel(0).copy(call.rel(0).getTraitSet(), ImmutableList.of(join));
-
-    // Convert the input expression into a MultiJoin
-    RelOptPlanner planner = call.getPlanner();
-    final HepPlanner hepPlanner =
-        new HepPlanner(multiJoinProgram, planner.getContext());
-    hepPlanner.setRoot(project);
-    RelNode best = hepPlanner.findBestExp();
-
-    if (best instanceof Project) {
-      best = ((Project) best).getInput();
-    }
-    if (!(best instanceof MultiJoin)) {
-      return;
-    }
-    apply(call, (MultiJoin) best, originalProject);
-  }
-
-  protected void apply(RelOptRuleCall call, MultiJoin join, Project originalProject) {
-    if (!isSupportedJoin(join)) {
-      return;
-    }
-    SortedMap<Integer, ImmutableBitSet> queryFilter = filterConditions(join);
-    if (queryFilter == null) {
-      return;
-    }
-    List<RelOptTable> queryTables = RelOptUtil.findAllTables(join);
-    Map<Integer, Pair<RelOptTable, RexInputRef>> queryFields =
-        originalFields(join, queryTables);
-    if (queryFields == null) {
-      return;
-    }
-
-    RelOptPlanner planner = call.getPlanner();
-    List<RelOptMaterialization> materializations =
-        planner instanceof VolcanoPlanner
-            ? ((VolcanoPlanner) planner).getMaterializations()
-            : ImmutableList.<RelOptMaterialization>of();
-    if (!materializations.isEmpty()) {
-      List<RelOptMaterialization> applicableMaterializations =
-          RelOptMaterializations.getApplicableMaterializations(join, materializations);
-
-      // Prepare a planner to convert views to MultiJoins
-      HepPlanner hepPlanner =
-          new HepPlanner(multiJoinProgram, planner.getContext());
-
-      for (RelOptMaterialization materialization : applicableMaterializations) {
-        // Skip over single table views
-        RelNode target = materialization.queryRel;
-        if (target instanceof TableScan
-            || (target instanceof Project
-                && ((Project) target).getInput() instanceof TableScan)) {
-          continue;
-        }
-
-        // Convert the view into a MultiJoin
-        hepPlanner.setRoot(target);
-        target = hepPlanner.findBestExp();
-        if (!(target instanceof Project)) { continue; }
-
-        Project viewProject = (Project) target;
-        if (!(viewProject.getInput() instanceof MultiJoin)) {
-          continue;
-        }
-        MultiJoin viewJoin = (MultiJoin) viewProject.getInput();
-        if (!isSupportedJoin(viewJoin)) {
-          continue;
-        }
-
-        List<RelOptTable> viewTables = RelOptUtil.findAllTables(viewJoin);
-
-        // Check that the same set of tables are in use
-        if (queryTables.size() != viewTables.size()
-            || !ImmutableSet.copyOf(queryTables).containsAll(viewTables)) {
-          continue;
-        }
-
-        // Extra the conditions and field from the view and ensure
-        // that they are all supported
-        SortedMap<Integer, ImmutableBitSet> viewFilter = filterConditions(viewJoin);
-        if (viewFilter == null) {
-          continue;
-        }
-        Map<Integer, Pair<RelOptTable, RexInputRef>> viewFields =
-            originalFields(viewJoin, viewTables);
-        if (viewFields == null) {
-          continue;
-        }
-
-        // If we fail to find one of the fields we are required
-        // to project, we can't use this view
-        List<RexNode> projects = materializedViewProjects(queryFields, queryFilter,
-            viewFields, originalProject);
-        if (projects.size() != originalProject.getNamedProjects().size()) {
-          continue;
-        }
-
-        final RelNode newNode = originalProject.copy(originalProject.getTraitSet(),
-                materialization.tableRel,
-                projects, originalProject.getRowType());
-        call.transformTo(newNode);
-      }
-    }
-  }
-
-  /**
-   * Checks that the join consists of either table scans or projects of scans
-   */
-  private boolean isSimpleProjects(MultiJoin join) {
-    for (RelNode input : join.getInputs()) {
-      if (!(input instanceof TableScan)
-            && !(input instanceof Project
-                && ((Project) input).getInput() instanceof TableScan)) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  private boolean isSupportedJoin(MultiJoin join) {
-    // We only support inner joins without post join filters over simple projects/scans
-    return !join.containsOuter() && join.getPostJoinFilter() == null && isSimpleProjects(join);
-  }
-
-  /**
-   * Produces a map from fields in a multijoin to references in the
-   * original tables referenced in the join
-   */
-  private Map<Integer, Pair<RelOptTable, RexInputRef>> originalFields(MultiJoin join,
-        List<RelOptTable> tables) {
-    List<ImmutableBitSet> projFields = join.getProjFields();
-    Map<Integer, Pair<RelOptTable, RexInputRef>> tableFields = new LinkedHashMap<>();
-    List<RelNode> inputs = join.getInputs();
-    int fieldNum = 0;
-    for (int i = 0; i < projFields.size(); i++) {
-      // Either get the project or construct a list projecting all fields
-      List<RexNode> projects;
-      if (inputs.get(i) instanceof Project) {
-        projects = ((Project) inputs.get(i)).getProjects();
-      } else {
-        assert inputs.get(i) instanceof TableScan;
-        List<RelDataTypeField> fields = inputs.get(i).getRowType().getFieldList();
-        projects = new ArrayList<>();
-        for (int j = 0; j < fields.size(); j++) {
-          projects.add(new RexInputRef(j, fields.get(j).getType()));
-        }
-      }
-
-      if (projFields.get(i) == null) { return null; }
-
-      int bit = projFields.get(i).nextSetBit(0);
-      while (bit != -1) {
-        // We currently only support rewriting of views with simple field references
-        if (!(projects.get(bit) instanceof RexInputRef)) {
-          return null;
-        }
-
-        tableFields.put(fieldNum, Pair.of(tables.get(i), (RexInputRef) projects.get(bit)));
-        fieldNum++;
-        bit = projFields.get(i).nextSetBit(bit + 1);
-      }
-    }
-
-    return tableFields;
-  }
-
-  /**
-   * If the node represents a field reference, get its index
-   */
-  private Integer getFieldIndex(RexNode operand) {
-    if (operand.isA(SqlKind.INPUT_REF)) {
-      return ((RexInputRef) operand).getIndex();
-    } else if (operand.isA(SqlKind.CAST)) {
-      return getFieldIndex(((RexCall) operand).getOperands().get(0));
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * Construct a map of equivalence classes of all columns
-   * in all tables used as input to the join
-   */
-  private SortedMap<Integer, ImmutableBitSet> filterConditions(MultiJoin join) {
-    SortedMap<Integer, ImmutableBitSet> equiv = new TreeMap<>();
-    RexNode filter = RexUtil.toCnf(join.getCluster().getRexBuilder(), join.getJoinFilter());
-    for (RexNode conjunct : RelOptUtil.conjunctions(filter)) {
-      List<RexNode> condition = RelOptUtil.disjunctions(conjunct);
-      if (condition.size() == 1 && condition.get(0).isA(SqlKind.EQUALS)) {
-        List<RexNode> operands = ((RexCall) condition.get(0)).getOperands();
-        Integer index1 = getFieldIndex(operands.get(0));
-        Integer index2 = getFieldIndex(operands.get(1));
-        if (index1 == null || index2 == null) {
-          // All operands to a condition must be field references or
-          // simple casts of field references
-          return null;
-        }
-        equiv.put(index1, ImmutableBitSet.of(index1, index2));
-      } else {
-        // We don't handle disjunctions or inequalities
-        return null;
-      }
-    }
-
-    equiv = ImmutableBitSet.closure(equiv);
-    return equiv;
-  }
-
-
-  /**
-   * Construct a list of projects we need on top of the materialized view
-   */
-  private List<RexNode> materializedViewProjects(
-        Map<Integer, Pair<RelOptTable, RexInputRef>> queryFields,
-        SortedMap<Integer, ImmutableBitSet> queryFilter,
-        Map<Integer, Pair<RelOptTable, RexInputRef>> viewFields,
-        Project originalProject) {
-    List<Pair<RelOptTable, RexInputRef>> viewFieldList =
-        Lists.newArrayList(viewFields.values().iterator());
-    List<Pair<RelOptTable, RexInputRef>> queryFieldList =
-        Lists.newArrayList(queryFields.values().iterator());
-    List<RexNode> projects = new ArrayList<>();
-    for (Map.Entry<Integer, Pair<RelOptTable, RexInputRef>> field
-          : queryFields.entrySet()) {
-      int fieldIndex = viewFieldList.indexOf(field.getValue());
-      if (fieldIndex == -1) {
-        // Check for equivalent fields in the view
-        ImmutableBitSet queryEquiv = queryFilter.get(field.getKey());
-        if (queryEquiv != null) {
-          for (Integer index : queryEquiv) {
-            fieldIndex = viewFieldList.indexOf(queryFieldList.get(index));
-            if (fieldIndex != -1) {
-              break;
-            }
-          }
-        }
-      }
-
-      if (fieldIndex == -1) {
-        break;
-      }
-
-      RelDataType type = field.getValue().right.getType();
-      RelDataType originalType = originalProject.getProjects().get(projects.size()).getType();
-      // if (originalType.equals(type)) {
-      if (!SqlTypeUtil.canCastFrom(originalType, type, false)) {
-        continue;
-      }
-      projects.add(new RexInputRef(fieldIndex, type));
-    }
-
-    return projects;
-  }
-}
-
-// End MaterializedViewJoinRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/27ca3109/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 bce7626..54ddaed 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -29,7 +29,6 @@ import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.RelReferentialConstraintImpl;
 import org.apache.calcite.rel.RelVisitor;
 import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.rules.MaterializedViewJoinRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rex.RexBuilder;
@@ -987,9 +986,7 @@ public class MaterializationTest {
         + "join \"depts\" using (\"deptno\") where \"empid\" = 1";
     final String m = "select \"empid\" \"deptno\" from \"emps\"\n"
         + "join \"depts\" using (\"deptno\")";
-    RuleSet rules = RuleSets.ofList(MaterializedViewJoinRule.INSTANCE_PROJECT,
-        MaterializedViewJoinRule.INSTANCE_TABLE_SCAN);
-    checkMaterializeWithRules(m, q, rules);
+    checkMaterialize(m, q);
   }
 
   @Test public void testUnionAll() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/27ca3109/site/_docs/materialized_views.md
----------------------------------------------------------------------
diff --git a/site/_docs/materialized_views.md b/site/_docs/materialized_views.md
index cae361b..2d4c682 100644
--- a/site/_docs/materialized_views.md
+++ b/site/_docs/materialized_views.md
@@ -58,27 +58,18 @@ This can accomplish a large number of rewritings.
 However, this approach is not scalable in the presence of complex views, e.g., views containing many join operators, 
 since it relies on the planner rules to create the equivalence between expressions.
 
-In turn, two alternative rules that attempt to match queries to views defined using arbitrary queries, 
-have been proposed. They are both based on the ideas of the [same paper](http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.95.113).
+In turn, an alternative rule that attempts to match queries to views defined using arbitrary queries 
+has been proposed.
 
-__MaterializedViewJoinRule__ is the first alternative. There are several limitations to the current implementation:
+{AbstractMaterializedViewRule} builds on the ideas presented [here](http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.95.113).
+The rule can rewrite expressions containing arbitrary chains of Join, Filter, and Project operators.
+Additionally, the rule can rewrite expressions rooted at an Aggregate operator, rolling aggregations up if necessary.
 
-1. The query defining the view must use only inner joins
-2. Only equality predicates are supported
-3. Predicates on tables used in the view must exactly match predicates in the query
-4. Rewriting is unoptimized and will attempt to match all views against each query
-
-These limitations are not fundamental the approach however and will hopefully be removed in the future.
-Note that the rule is currently disabled by default.
-To make use of the rule, {MaterializedViewJoinRule.INSTANCE_PROJECT} and {MaterializedViewJoinRule.INSTANCE_TABLE_SCAN} need to be added to the planner.
-
-__AbstractMaterializedViewRule__ is the second alternative. It builds on the same ideas but it attempts to be more generic.
-In particular, some of the limitations of the previous rule, such as number `2.` and `3.`, do not exist for this rule.
-Additionally, the rule will be able to rewrite expressions rooted at an Aggregate operator, rolling aggregations up if necessary.
-
-However, this rule still presents some limitations too. In addition to `1.` and `4.` above, the rule presents following
+However, this rule still presents some limitations. In particular, the rule presents the following
 shortcomings that we plan to address with follow-up extensions:
 
+* Rewriting is unoptimized and will attempt to match all views against each query.
+* The query defining the view must use only inner joins.
 * It does not produce rewritings using Union operators, e.g., a given query could be partially answered from the
 {mv} (year = 2014) and from the query (not(year=2014)). This can be useful if {mv} is stored in a system such as
 Druid.


[11/11] calcite git commit: [CALCITE-1456] Change SubstitutionVisitor to use generic RelBuilder instead of Logical instances of the operators when possible

Posted by jc...@apache.org.
[CALCITE-1456] Change SubstitutionVisitor to use generic RelBuilder instead of Logical instances of the operators when possible


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

Branch: refs/heads/master
Commit: e9d0ca6731b2f5ec33b9270b5ffcaaf9e4eb8537
Parents: 9a691a7
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Apr 26 19:09:19 2017 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Apr 26 20:05:03 2017 +0100

----------------------------------------------------------------------
 .../MaterializedViewSubstitutionVisitor.java    |  6 ++
 .../calcite/plan/SubstitutionVisitor.java       | 32 +++++---
 .../apache/calcite/rel/mutable/MutableRels.java | 80 +++++++++++---------
 3 files changed, 75 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/e9d0ca67/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java b/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java
index c1e0e37..a2ff5f4 100644
--- a/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/MaterializedViewSubstitutionVisitor.java
@@ -25,6 +25,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.tools.RelBuilderFactory;
 
 import com.google.common.collect.ImmutableList;
 
@@ -46,6 +47,11 @@ public class MaterializedViewSubstitutionVisitor extends SubstitutionVisitor {
     super(target_, query_, EXTENDED_RULES);
   }
 
+  public MaterializedViewSubstitutionVisitor(RelNode target_, RelNode query_,
+      RelBuilderFactory relBuilderFactory) {
+    super(target_, query_, EXTENDED_RULES, relBuilderFactory);
+  }
+
   public List<RelNode> go(RelNode replacement_) {
     return super.go(replacement_);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/e9d0ca67/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 f7bf106..fca0c56 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -21,6 +21,7 @@ import org.apache.calcite.prepare.CalcitePrepareImpl;
 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.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.mutable.Holder;
@@ -46,6 +47,8 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -127,6 +130,11 @@ public class SubstitutionVisitor {
           AggregateToAggregateUnifyRule.INSTANCE,
           AggregateOnProjectToAggregateUnifyRule.INSTANCE);
 
+  /**
+   * Factory for a builder for relational expressions.
+   */
+  protected final RelBuilder relBuilder;
+
   private final ImmutableList<UnifyRule> rules;
   private final Map<Pair<Class, Class>, List<UnifyRule>> ruleMap =
       new HashMap<>();
@@ -157,12 +165,17 @@ public class SubstitutionVisitor {
 
   /** Creates a SubstitutionVisitor with the default rule set. */
   public SubstitutionVisitor(RelNode target_, RelNode query_) {
-    this(target_, query_, DEFAULT_RULES);
+    this(target_, query_, DEFAULT_RULES, RelFactories.LOGICAL_BUILDER);
   }
 
-  /** Creates a SubstitutionVisitor. */
+  /** Creates a SubstitutionVisitor with the default logical builder. */
   public SubstitutionVisitor(RelNode target_, RelNode query_,
       ImmutableList<UnifyRule> rules) {
+    this(target_, query_, rules, RelFactories.LOGICAL_BUILDER);
+  }
+
+  public SubstitutionVisitor(RelNode target_, RelNode query_,
+      ImmutableList<UnifyRule> rules, RelBuilderFactory relBuilderFactory) {
     this.cluster = target_.getCluster();
     final RexExecutor executor =
         Util.first(cluster.getPlanner().getExecutor(), RexUtil.EXECUTOR);
@@ -170,6 +183,7 @@ public class SubstitutionVisitor {
     this.rules = rules;
     this.query = Holder.of(MutableRels.toMutable(query_));
     this.target = MutableRels.toMutable(target_);
+    this.relBuilder = relBuilderFactory.create(cluster, null);
     final Set<MutableRel> parents = Sets.newIdentityHashSet();
     final List<MutableRel> allNodes = new ArrayList<>();
     final MutableRelVisitor visitor =
@@ -395,7 +409,7 @@ public class SubstitutionVisitor {
           + "\nnode:\n"
           + node.deep());
     }
-    return MutableRels.fromMutable(node);
+    return MutableRels.fromMutable(node, relBuilder);
   }
 
   /**
@@ -412,8 +426,8 @@ public class SubstitutionVisitor {
       return ImmutableList.of();
     }
     List<RelNode> sub = Lists.newArrayList();
-    sub.add(MutableRels.fromMutable(query.getInput()));
-    reverseSubstitute(query, matches, sub, 0, matches.size());
+    sub.add(MutableRels.fromMutable(query.getInput(), relBuilder));
+    reverseSubstitute(relBuilder, query, matches, sub, 0, matches.size());
     return sub;
   }
 
@@ -594,19 +608,19 @@ public class SubstitutionVisitor {
     }
   }
 
-  private static void reverseSubstitute(Holder query,
+  private static void reverseSubstitute(RelBuilder relBuilder, Holder query,
       List<List<Replacement>> matches, List<RelNode> sub,
       int replaceCount, int maxCount) {
     if (matches.isEmpty()) {
       return;
     }
     final List<List<Replacement>> rem = matches.subList(1, matches.size());
-    reverseSubstitute(query, rem, sub, replaceCount, maxCount);
+    reverseSubstitute(relBuilder, query, rem, sub, replaceCount, maxCount);
     undoReplacement(matches.get(0));
     if (++replaceCount < maxCount) {
-      sub.add(MutableRels.fromMutable(query.getInput()));
+      sub.add(MutableRels.fromMutable(query.getInput(), relBuilder));
     }
-    reverseSubstitute(query, rem, sub, replaceCount, maxCount);
+    reverseSubstitute(relBuilder, query, rem, sub, replaceCount, maxCount);
     redoReplacement(matches.get(0));
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e9d0ca67/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java b/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
index ed1f1b1..b07da0e 100644
--- a/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
+++ b/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
@@ -30,6 +30,7 @@ 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.RelFactories;
 import org.apache.calcite.rel.core.Sample;
 import org.apache.calcite.rel.core.SemiJoin;
 import org.apache.calcite.rel.core.Sort;
@@ -40,24 +41,18 @@ import org.apache.calcite.rel.core.Uncollect;
 import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rel.core.Window;
-import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.logical.LogicalCorrelate;
 import org.apache.calcite.rel.logical.LogicalExchange;
-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;
-import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
 import org.apache.calcite.rel.logical.LogicalTableModify;
-import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.logical.LogicalWindow;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
 
@@ -177,97 +172,114 @@ public abstract class MutableRels {
   }
 
   public static RelNode fromMutable(MutableRel node) {
+    return fromMutable(node, RelFactories.LOGICAL_BUILDER.create(node.cluster, null));
+  }
+
+  public static RelNode fromMutable(MutableRel node, RelBuilder relBuilder) {
     switch (node.type) {
     case TABLE_SCAN:
     case VALUES:
       return ((MutableLeafRel) node).rel;
     case PROJECT:
       final MutableProject project = (MutableProject) node;
-      return LogicalProject.create(
-          fromMutable(project.input), project.projects, project.rowType);
+      relBuilder.push(fromMutable(project.input, relBuilder));
+      relBuilder.project(project.projects, project.rowType.getFieldNames(), true);
+      return relBuilder.build();
     case FILTER:
       final MutableFilter filter = (MutableFilter) node;
-      return LogicalFilter.create(fromMutable(filter.input),
-          filter.condition);
+      relBuilder.push(fromMutable(filter.input, relBuilder));
+      relBuilder.filter(filter.condition);
+      return relBuilder.build();
     case AGGREGATE:
       final MutableAggregate aggregate = (MutableAggregate) node;
-      return LogicalAggregate.create(fromMutable(aggregate.input),
-          aggregate.indicator, aggregate.groupSet, aggregate.groupSets,
+      relBuilder.push(fromMutable(aggregate.input, relBuilder));
+      relBuilder.aggregate(
+          relBuilder.groupKey(aggregate.groupSet, aggregate.indicator, aggregate.groupSets),
           aggregate.aggCalls);
+      return relBuilder.build();
     case SORT:
       final MutableSort sort = (MutableSort) node;
-      return LogicalSort.create(fromMutable(sort.input), sort.collation,
+      return LogicalSort.create(fromMutable(sort.input, relBuilder), sort.collation,
           sort.offset, sort.fetch);
     case CALC:
       final MutableCalc calc = (MutableCalc) node;
-      return LogicalCalc.create(fromMutable(calc.input), calc.program);
+      return LogicalCalc.create(fromMutable(calc.input, relBuilder), calc.program);
     case EXCHANGE:
       final MutableExchange exchange = (MutableExchange) node;
       return LogicalExchange.create(
-          fromMutable(exchange.getInput()), exchange.distribution);
+          fromMutable(exchange.getInput(), relBuilder), exchange.distribution);
     case COLLECT: {
       final MutableCollect collect = (MutableCollect) node;
-      final RelNode child = fromMutable(collect.getInput());
+      final RelNode child = fromMutable(collect.getInput(), relBuilder);
       return new Collect(collect.cluster, child.getTraitSet(), child, collect.fieldName);
     }
     case UNCOLLECT: {
       final MutableUncollect uncollect = (MutableUncollect) node;
-      final RelNode child = fromMutable(uncollect.getInput());
+      final RelNode child = fromMutable(uncollect.getInput(), relBuilder);
       return Uncollect.create(child.getTraitSet(), child, uncollect.withOrdinality);
     }
     case WINDOW: {
       final MutableWindow window = (MutableWindow) node;
-      final RelNode child = fromMutable(window.getInput());
+      final RelNode child = fromMutable(window.getInput(), relBuilder);
       return LogicalWindow.create(child.getTraitSet(),
           child, window.constants, window.rowType, window.groups);
     }
     case TABLE_MODIFY:
       final MutableTableModify modify = (MutableTableModify) node;
       return LogicalTableModify.create(modify.table, modify.catalogReader,
-          fromMutable(modify.getInput()), modify.operation, modify.updateColumnList,
+          fromMutable(modify.getInput(), relBuilder), modify.operation, modify.updateColumnList,
           modify.sourceExpressionList, modify.flattened);
     case SAMPLE:
       final MutableSample sample = (MutableSample) node;
-      return new Sample(sample.cluster, fromMutable(sample.getInput()), sample.params);
+      return new Sample(sample.cluster, fromMutable(sample.getInput(), relBuilder), sample.params);
     case TABLE_FUNCTION_SCAN:
       final MutableTableFunctionScan tableFunctionScan = (MutableTableFunctionScan) node;
       return LogicalTableFunctionScan.create(tableFunctionScan.cluster,
-          fromMutables(tableFunctionScan.getInputs()), tableFunctionScan.rexCall,
+          fromMutables(tableFunctionScan.getInputs(), relBuilder), tableFunctionScan.rexCall,
           tableFunctionScan.elementType, tableFunctionScan.rowType,
           tableFunctionScan.columnMappings);
     case JOIN:
       final MutableJoin join = (MutableJoin) node;
-      return LogicalJoin.create(fromMutable(join.getLeft()), fromMutable(join.getRight()),
-          join.condition, join.variablesSet, join.joinType);
+      relBuilder.push(fromMutable(join.getLeft(), relBuilder));
+      relBuilder.push(fromMutable(join.getRight(), relBuilder));
+      relBuilder.join(join.joinType, join.condition, join.variablesSet);
+      return relBuilder.build();
     case SEMIJOIN:
       final MutableSemiJoin semiJoin = (MutableSemiJoin) node;
-      return SemiJoin.create(fromMutable(semiJoin.getLeft()),
-          fromMutable(semiJoin.getRight()), semiJoin.condition,
-          semiJoin.leftKeys, semiJoin.rightKeys);
+      relBuilder.push(fromMutable(semiJoin.getLeft(), relBuilder));
+      relBuilder.push(fromMutable(semiJoin.getRight(), relBuilder));
+      relBuilder.semiJoin(semiJoin.condition);
+      return relBuilder.build();
     case CORRELATE:
       final MutableCorrelate correlate = (MutableCorrelate) node;
-      return LogicalCorrelate.create(fromMutable(correlate.getLeft()),
-          fromMutable(correlate.getRight()), correlate.correlationId,
+      return LogicalCorrelate.create(fromMutable(correlate.getLeft(), relBuilder),
+          fromMutable(correlate.getRight(), relBuilder), correlate.correlationId,
           correlate.requiredColumns, correlate.joinType);
     case UNION:
       final MutableUnion union = (MutableUnion) node;
-      return LogicalUnion.create(MutableRels.fromMutables(union.inputs), union.all);
+      relBuilder.pushAll(MutableRels.fromMutables(union.inputs, relBuilder));
+      relBuilder.union(union.all, union.inputs.size());
+      return relBuilder.build();
     case MINUS:
       final MutableMinus minus = (MutableMinus) node;
-      return LogicalMinus.create(MutableRels.fromMutables(minus.inputs), minus.all);
+      relBuilder.pushAll(MutableRels.fromMutables(minus.inputs, relBuilder));
+      relBuilder.minus(minus.all, minus.inputs.size());
+      return relBuilder.build();
     case INTERSECT:
       final MutableIntersect intersect = (MutableIntersect) node;
-      return LogicalIntersect.create(MutableRels.fromMutables(intersect.inputs), intersect.all);
+      relBuilder.pushAll(MutableRels.fromMutables(intersect.inputs, relBuilder));
+      relBuilder.intersect(intersect.all, intersect.inputs.size());
+      return relBuilder.build();
     default:
       throw new AssertionError(node.deep());
     }
   }
 
-  private static List<RelNode> fromMutables(List<MutableRel> nodes) {
+  private static List<RelNode> fromMutables(List<MutableRel> nodes, final RelBuilder relBuilder) {
     return Lists.transform(nodes,
         new Function<MutableRel, RelNode>() {
           public RelNode apply(MutableRel mutableRel) {
-            return fromMutable(mutableRel);
+            return fromMutable(mutableRel, relBuilder);
           }
         });
   }


[04/11] calcite git commit: [CALCITE-1682] New metadata providers for expression column origin and all predicates in plan

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 234b666..dbc230a 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -36,11 +36,16 @@ 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.Filter;
 import org.apache.calcite.rel.core.Join;
 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.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.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalExchange;
 import org.apache.calcite.rel.logical.LogicalFilter;
@@ -65,9 +70,13 @@ import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 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.RexNode;
+import org.apache.calcite.rex.RexTableInputRef;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.Frameworks;
@@ -80,6 +89,7 @@ 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 com.google.common.collect.Multimap;
 
 import org.hamcrest.CoreMatchers;
 import org.hamcrest.CustomTypeSafeMatcher;
@@ -91,17 +101,27 @@ import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
+import static org.hamcrest.CoreMatchers.endsWith;
 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.hamcrest.CoreMatchers.notNullValue;
 import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -133,6 +153,8 @@ public class RelMetadataTest extends SqlToRelTestBase {
 
   private static final double DEPT_SIZE = 4d;
 
+  private static final String EMP_QNAME = "[CATALOG, SALES, EMP]";
+
   //~ Methods ----------------------------------------------------------------
 
   private static Matcher<? super Number> nearTo(Number v, Number epsilon) {
@@ -1481,6 +1503,631 @@ public class RelMetadataTest extends SqlToRelTestBase {
     assertThat(RelMdUtil.linear(12, 0, 10, 100, 200), is(200d));
   }
 
+  @Test public void testExpressionLineageStar() {
+    // All columns in output
+    final RelNode tableRel = convertSql("select * from emp");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(4, tableRel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(tableRel, ref);
+    final String inputRef = RexInputRef.of(4, tableRel.getRowType().getFieldList()).toString();
+    assertThat(r.size(), is(1));
+    final String resultString = r.iterator().next().toString();
+    assertThat(resultString, startsWith(EMP_QNAME));
+    assertThat(resultString, endsWith(inputRef));
+  }
+
+  @Test public void testExpressionLineageTwoColumns() {
+    // mgr is column 3 in catalog.sales.emp
+    // deptno is column 7 in catalog.sales.emp
+    final RelNode rel = convertSql("select mgr, deptno from emp");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref1 = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r1 = mq.getExpressionLineage(rel, ref1);
+    assertThat(r1.size(), is(1));
+    final RexTableInputRef result1 = (RexTableInputRef) r1.iterator().next();
+    assertThat(result1.getQualifiedName(), is(EMP_QNAME));
+    assertThat(result1.getIndex(), is(3));
+
+    final RexNode ref2 = RexInputRef.of(1, rel.getRowType().getFieldList());
+    final Set<RexNode> r2 = mq.getExpressionLineage(rel, ref2);
+    assertThat(r2.size(), is(1));
+    final RexTableInputRef result2 = (RexTableInputRef) r2.iterator().next();
+    assertThat(result2.getQualifiedName(), is(EMP_QNAME));
+    assertThat(result2.getIndex(), is(7));
+
+    assertThat(result1.getIdentifier(), is(result2.getIdentifier()));
+  }
+
+  @Test public void testExpressionLineageTwoColumnsSwapped() {
+    // deptno is column 7 in catalog.sales.emp
+    // mgr is column 3 in catalog.sales.emp
+    final RelNode rel = convertSql("select deptno, mgr from emp");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref1 = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r1 = mq.getExpressionLineage(rel, ref1);
+    assertThat(r1.size(), is(1));
+    final RexTableInputRef result1 = (RexTableInputRef) r1.iterator().next();
+    assertThat(result1.getQualifiedName(), is(EMP_QNAME));
+    assertThat(result1.getIndex(), is(7));
+
+    final RexNode ref2 = RexInputRef.of(1, rel.getRowType().getFieldList());
+    final Set<RexNode> r2 = mq.getExpressionLineage(rel, ref2);
+    assertThat(r2.size(), is(1));
+    final RexTableInputRef result2 = (RexTableInputRef) r2.iterator().next();
+    assertThat(result2.getQualifiedName(), is(EMP_QNAME));
+    assertThat(result2.getIndex(), is(3));
+
+    assertThat(result1.getIdentifier(), is(result2.getIdentifier()));
+  }
+
+  @Test public void testExpressionLineageCombineTwoColumns() {
+    // empno is column 0 in catalog.sales.emp
+    // deptno is column 7 in catalog.sales.emp
+    final RelNode rel = convertSql("select empno + deptno from emp");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+
+    assertThat(r.size(), is(1));
+    final RexNode result = r.iterator().next();
+    assertThat(result.getKind(), is(SqlKind.PLUS));
+    final RexCall call = (RexCall) result;
+    assertThat(call.getOperands().size(), is(2));
+    final RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
+    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertThat(inputRef1.getIndex(), is(0));
+    final RexTableInputRef inputRef2 = (RexTableInputRef) call.getOperands().get(1);
+    assertThat(inputRef2.getQualifiedName(), is(EMP_QNAME));
+    assertThat(inputRef2.getIndex(), is(7));
+    assertThat(inputRef1.getIdentifier(), is(inputRef2.getIdentifier()));
+  }
+
+  @Test public void testExpressionLineageInnerJoinLeft() {
+    // ename is column 1 in catalog.sales.emp
+    final RelNode rel = convertSql("select ename from emp,dept");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+    assertThat(r.size(), is(1));
+    final RexTableInputRef result = (RexTableInputRef) r.iterator().next();
+    assertThat(result.getQualifiedName(), is(EMP_QNAME));
+    assertThat(result.getIndex(), is(1));
+  }
+
+  @Test public void testExpressionLineageInnerJoinRight() {
+    // ename is column 0 in catalog.sales.bonus
+    final RelNode rel = convertSql("select bonus.ename from emp join bonus using (ename)");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+    assertThat(r.size(), is(1));
+    final RexTableInputRef result = (RexTableInputRef) r.iterator().next();
+    assertThat(result.getQualifiedName(), is("[CATALOG, SALES, BONUS]"));
+    assertThat(result.getIndex(), is(0));
+  }
+
+  @Test public void testExpressionLineageSelfJoin() {
+    // deptno is column 7 in catalog.sales.emp
+    // sal is column 5 in catalog.sales.emp
+    final RelNode rel = convertSql("select a.deptno, b.sal from (select * from emp limit 7) as a\n"
+        + "inner join (select * from emp limit 2) as b\n"
+        + "on a.deptno = b.deptno");
+    final RelNode tableRel = convertSql("select * from emp");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref1 = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r1 = mq.getExpressionLineage(rel, ref1);
+    final String inputRef1 = RexInputRef.of(7, tableRel.getRowType().getFieldList()).toString();
+    assertThat(r1.size(), is(1));
+    final String resultString1 = r1.iterator().next().toString();
+    assertThat(resultString1, startsWith(EMP_QNAME));
+    assertThat(resultString1, endsWith(inputRef1));
+
+    final RexNode ref2 = RexInputRef.of(1, rel.getRowType().getFieldList());
+    final Set<RexNode> r2 = mq.getExpressionLineage(rel, ref2);
+    final String inputRef2 = RexInputRef.of(5, tableRel.getRowType().getFieldList()).toString();
+    assertThat(r2.size(), is(1));
+    final String resultString2 = r2.iterator().next().toString();
+    assertThat(resultString2, startsWith(EMP_QNAME));
+    assertThat(resultString2, endsWith(inputRef2));
+
+    assertThat(((RexTableInputRef) r1.iterator().next()).getIdentifier(),
+        not(((RexTableInputRef) r2.iterator().next()).getIdentifier()));
+  }
+
+  @Test public void testExpressionLineageOuterJoin() {
+    // lineage cannot be determined
+    final RelNode rel = convertSql("select name as dname from emp left outer join dept"
+        + " on emp.deptno = dept.deptno");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+    assertNull(r);
+  }
+
+  @Test public void testExpressionLineageFilter() {
+    // ename is column 1 in catalog.sales.emp
+    final RelNode rel = convertSql("select ename from emp where deptno = 10");
+    final RelNode tableRel = convertSql("select * from emp");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+    final String inputRef = RexInputRef.of(1, tableRel.getRowType().getFieldList()).toString();
+    assertThat(r.size(), is(1));
+    final String resultString = r.iterator().next().toString();
+    assertThat(resultString, startsWith(EMP_QNAME));
+    assertThat(resultString, endsWith(inputRef));
+  }
+
+  @Test public void testExpressionLineageAggregateGroupColumn() {
+    // deptno is column 7 in catalog.sales.emp
+    final RelNode rel = convertSql("select deptno, count(*) from emp where deptno > 10 "
+        + "group by deptno having count(*) = 0");
+    final RelNode tableRel = convertSql("select * from emp");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+    final String inputRef = RexInputRef.of(7, tableRel.getRowType().getFieldList()).toString();
+    assertThat(r.size(), is(1));
+    final String resultString = r.iterator().next().toString();
+    assertThat(resultString, startsWith(EMP_QNAME));
+    assertThat(resultString, endsWith(inputRef));
+  }
+
+  @Test public void testExpressionLineageAggregateAggColumn() {
+    // lineage cannot be determined
+    final RelNode rel = convertSql("select deptno, count(*) from emp where deptno > 10 "
+        + "group by deptno having count(*) = 0");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(1, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+    assertNull(r);
+  }
+
+  @Test public void testExpressionLineageUnion() {
+    // sal is column 5 in catalog.sales.emp
+    final RelNode rel = convertSql("select sal from (\n"
+        + "  select * from emp union all select * from emp) "
+        + "where deptno = 10");
+    final RelNode tableRel = convertSql("select * from emp");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+    final String inputRef = RexInputRef.of(5, tableRel.getRowType().getFieldList()).toString();
+    assertThat(r.size(), is(2));
+    for (RexNode result : r) {
+      final String resultString = result.toString();
+      assertThat(resultString, startsWith(EMP_QNAME));
+      assertThat(resultString, endsWith(inputRef));
+    }
+
+    Iterator<RexNode> it = r.iterator();
+    assertThat(((RexTableInputRef) it.next()).getIdentifier(),
+        not(((RexTableInputRef) it.next()).getIdentifier()));
+  }
+
+  @Test public void testExpressionLineageMultiUnion() {
+    // empno is column 0 in catalog.sales.emp
+    // sal is column 5 in catalog.sales.emp
+    final RelNode rel = convertSql("select a.empno + b.sal from \n"
+        + " (select empno, ename from emp,dept) a join "
+        + " (select * from emp union all select * from emp) b \n"
+        + " on a.empno = b.empno \n"
+        + " where b.deptno = 10");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+
+    // With the union, we should get two origins
+    // The first one should be the same one: join
+    // The second should come from each union input
+    final Set<String> set = new HashSet<>();
+    assertThat(r.size(), is(2));
+    for (RexNode result : r) {
+      assertThat(result.getKind(), is(SqlKind.PLUS));
+      final RexCall call = (RexCall) result;
+      assertThat(call.getOperands().size(), is(2));
+      final RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
+      assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+      // Add join alpha to set
+      set.add(inputRef1.getQualifiedName());
+      assertThat(inputRef1.getIndex(), is(0));
+      final RexTableInputRef inputRef2 = (RexTableInputRef) call.getOperands().get(1);
+      assertThat(inputRef2.getQualifiedName(), is(EMP_QNAME));
+      assertThat(inputRef2.getIndex(), is(5));
+      assertThat(inputRef1.getIdentifier(), not(inputRef2.getIdentifier()));
+    }
+    assertThat(set.size(), is(1));
+  }
+
+  @Test public void testExpressionLineageValues() {
+    // lineage cannot be determined
+    final RelNode rel = convertSql("select * from (values (1), (2)) as t(c)");
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final RexNode ref = RexInputRef.of(0, rel.getRowType().getFieldList());
+    final Set<RexNode> r = mq.getExpressionLineage(rel, ref);
+    assertNull(r);
+  }
+
+  @Test public void testAllPredicates() {
+    final Project rel = (Project) convertSql("select * from emp, dept");
+    final Join join = (Join) rel.getInput();
+    final RelOptTable empTable = join.getInput(0).getTable();
+    final RelOptTable deptTable = join.getInput(1).getTable();
+    Frameworks.withPlanner(
+        new Frameworks.PlannerAction<Void>() {
+          public Void apply(RelOptCluster cluster,
+              RelOptSchema relOptSchema,
+              SchemaPlus rootSchema) {
+            checkAllPredicates(cluster, empTable, deptTable);
+            return null;
+          }
+        });
+  }
+
+  private void checkAllPredicates(RelOptCluster cluster, RelOptTable empTable,
+      RelOptTable deptTable) {
+    final RelBuilder relBuilder = RelBuilder.proto().create(cluster, null);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+
+    final LogicalTableScan empScan = LogicalTableScan.create(cluster, empTable);
+    relBuilder.push(empScan);
+
+    RelOptPredicateList predicates =
+        mq.getAllPredicates(empScan);
+    assertThat(predicates.pulledUpPredicates.isEmpty(), is(true));
+
+    relBuilder.filter(
+        relBuilder.equals(relBuilder.field("EMPNO"),
+            relBuilder.literal(BigDecimal.ONE)));
+
+    final RelNode filter = relBuilder.peek();
+    predicates = mq.getAllPredicates(filter);
+    assertThat(predicates.pulledUpPredicates.size(), is(1));
+    RexCall call = (RexCall) predicates.pulledUpPredicates.get(0);
+    assertThat(call.getOperands().size(), is(2));
+    RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
+    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertThat(inputRef1.getIndex(), is(0));
+
+    final LogicalTableScan deptScan =
+        LogicalTableScan.create(cluster, deptTable);
+    relBuilder.push(deptScan);
+
+    relBuilder.join(JoinRelType.INNER,
+        relBuilder.equals(relBuilder.field(2, 0, "DEPTNO"),
+            relBuilder.field(2, 1, "DEPTNO")));
+
+    relBuilder.project(relBuilder.field("DEPTNO"));
+    final RelNode project = relBuilder.peek();
+    predicates = mq.getAllPredicates(project);
+    assertThat(predicates.pulledUpPredicates.size(), is(2));
+    // From Filter
+    call = (RexCall) predicates.pulledUpPredicates.get(0);
+    assertThat(call.getOperands().size(), is(2));
+    inputRef1 = (RexTableInputRef) call.getOperands().get(0);
+    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertThat(inputRef1.getIndex(), is(0));
+    // From Join
+    call = (RexCall) predicates.pulledUpPredicates.get(1);
+    assertThat(call.getOperands().size(), is(2));
+    inputRef1 = (RexTableInputRef) call.getOperands().get(0);
+    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertThat(inputRef1.getIndex(), is(7));
+    RexTableInputRef inputRef2 = (RexTableInputRef) call.getOperands().get(1);
+    assertThat(inputRef2.getQualifiedName(), is("[CATALOG, SALES, DEPT]"));
+    assertThat(inputRef2.getIndex(), is(0));
+  }
+
+  @Test public void testAllPredicatesAggregate1() {
+    final String sql = "select a, max(b) from (\n"
+        + "  select empno as a, sal as b from emp where empno = 5)subq\n"
+        + "group by a";
+    final RelNode rel = convertSql(sql);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    RelOptPredicateList inputSet = mq.getAllPredicates(rel);
+    ImmutableList<RexNode> pulledUpPredicates = inputSet.pulledUpPredicates;
+    assertThat(pulledUpPredicates.size(), is(1));
+    RexCall call = (RexCall) pulledUpPredicates.get(0);
+    assertThat(call.getOperands().size(), is(2));
+    final RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
+    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertThat(inputRef1.getIndex(), is(0));
+    final RexLiteral constant = (RexLiteral) call.getOperands().get(1);
+    assertThat(constant.toString(), is("5"));
+  }
+
+  @Test public void testAllPredicatesAggregate2() {
+    final String sql = "select * from (select a, max(b) from (\n"
+        + "  select empno as a, sal as b from emp)subq\n"
+        + "group by a) \n"
+        + "where a = 5";
+    final RelNode rel = convertSql(sql);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    RelOptPredicateList inputSet = mq.getAllPredicates(rel);
+    ImmutableList<RexNode> pulledUpPredicates = inputSet.pulledUpPredicates;
+    assertThat(pulledUpPredicates.size(), is(1));
+    RexCall call = (RexCall) pulledUpPredicates.get(0);
+    assertThat(call.getOperands().size(), is(2));
+    final RexTableInputRef inputRef1 = (RexTableInputRef) call.getOperands().get(0);
+    assertThat(inputRef1.getQualifiedName(), is(EMP_QNAME));
+    assertThat(inputRef1.getIndex(), is(0));
+    final RexLiteral constant = (RexLiteral) call.getOperands().get(1);
+    assertThat(constant.toString(), is("5"));
+  }
+
+  @Test public void testAllPredicatesAggregate3() {
+    final String sql = "select * from (select a, max(b) as b from (\n"
+        + "  select empno as a, sal as b from emp)subq\n"
+        + "group by a) \n"
+        + "where b = 5";
+    final RelNode rel = convertSql(sql);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    RelOptPredicateList inputSet = mq.getAllPredicates(rel);
+    // Filter on aggregate, we cannot infer lineage
+    assertNull(inputSet);
+  }
+
+  private void checkNodeTypeCount(String sql, Map<Class<? extends RelNode>, Integer> expected) {
+    final RelNode rel = convertSql(sql);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final Multimap<Class<? extends RelNode>, RelNode> result = mq.getNodeTypes(rel);
+    assertThat(result, notNullValue());
+    final Map<Class<? extends RelNode>, Integer> resultCount = new HashMap<>();
+    for (Entry<Class<? extends RelNode>, Collection<RelNode>> e : result.asMap().entrySet()) {
+      resultCount.put(e.getKey(), e.getValue().size());
+    }
+    assertEquals(expected, resultCount);
+  }
+
+  @Test public void testNodeTypeCountEmp() {
+    final String sql = "select * from emp";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountDept() {
+    final String sql = "select * from dept";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountValues() {
+    final String sql = "select * from (values (1), (2)) as t(c)";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(Values.class, 1);
+    expected.put(Project.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountCartesian() {
+    final String sql = "select * from emp,dept";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Join.class, 1);
+    expected.put(Project.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountJoin() {
+    final String sql = "select * from emp\n"
+        + "inner join dept on emp.deptno = dept.deptno";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Join.class, 1);
+    expected.put(Project.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountJoinFinite() {
+    final String sql = "select * from (select * from emp limit 14) as emp\n"
+        + "inner join (select * from dept limit 4) as dept\n"
+        + "on emp.deptno = dept.deptno";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Join.class, 1);
+    expected.put(Project.class, 3);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountJoinEmptyFinite() {
+    final String sql = "select * from (select * from emp limit 0) as emp\n"
+        + "inner join (select * from dept limit 4) as dept\n"
+        + "on emp.deptno = dept.deptno";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Join.class, 1);
+    expected.put(Project.class, 3);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountLeftJoinEmptyFinite() {
+    final String sql = "select * from (select * from emp limit 0) as emp\n"
+        + "left join (select * from dept limit 4) as dept\n"
+        + "on emp.deptno = dept.deptno";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Join.class, 1);
+    expected.put(Project.class, 3);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountRightJoinEmptyFinite() {
+    final String sql = "select * from (select * from emp limit 0) as emp\n"
+        + "right join (select * from dept limit 4) as dept\n"
+        + "on emp.deptno = dept.deptno";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Join.class, 1);
+    expected.put(Project.class, 3);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountJoinFiniteEmpty() {
+    final String sql = "select * from (select * from emp limit 7) as emp\n"
+        + "inner join (select * from dept limit 0) as dept\n"
+        + "on emp.deptno = dept.deptno";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Join.class, 1);
+    expected.put(Project.class, 3);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountJoinEmptyEmpty() {
+    final String sql = "select * from (select * from emp limit 0) as emp\n"
+        + "inner join (select * from dept limit 0) as dept\n"
+        + "on emp.deptno = dept.deptno";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Join.class, 1);
+    expected.put(Project.class, 3);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountUnion() {
+    final String sql = "select ename from emp\n"
+        + "union all\n"
+        + "select name from dept";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Project.class, 2);
+    expected.put(Union.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountUnionOnFinite() {
+    final String sql = "select ename from (select * from emp limit 100)\n"
+        + "union all\n"
+        + "select name from (select * from dept limit 40)";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Union.class, 1);
+    expected.put(Project.class, 4);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountMinusOnFinite() {
+    final String sql = "select ename from (select * from emp limit 100)\n"
+        + "except\n"
+        + "select name from (select * from dept limit 40)";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 2);
+    expected.put(Minus.class, 1);
+    expected.put(Project.class, 4);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountFilter() {
+    final String sql = "select * from emp where ename='Mathilda'";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 1);
+    expected.put(Filter.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountSort() {
+    final String sql = "select * from emp order by ename";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 1);
+    expected.put(Sort.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountSortLimit() {
+    final String sql = "select * from emp order by ename limit 10";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 1);
+    expected.put(Sort.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountSortLimitOffset() {
+    final String sql = "select * from emp order by ename limit 10 offset 5";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 1);
+    expected.put(Sort.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountSortLimitOffsetOnFinite() {
+    final String sql = "select * from (select * from emp limit 12)\n"
+        + "order by ename limit 20 offset 5";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 2);
+    expected.put(Sort.class, 2);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountAggregate() {
+    final String sql = "select deptno from emp group by deptno";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 1);
+    expected.put(Aggregate.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountAggregateGroupingSets() {
+    final String sql = "select deptno from emp\n"
+        + "group by grouping sets ((deptno), (ename, deptno))";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 3);
+    expected.put(Aggregate.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountAggregateEmptyKeyOnEmptyTable() {
+    final String sql = "select count(*) from (select * from emp limit 0)";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 2);
+    expected.put(Aggregate.class, 1);
+    expected.put(Sort.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
+  @Test public void testNodeTypeCountFilterAggregateEmptyKey() {
+    final String sql = "select count(*) from emp where 1 = 0";
+    final Map<Class<? extends RelNode>, Integer> expected = new HashMap<>();
+    expected.put(TableScan.class, 1);
+    expected.put(Project.class, 1);
+    expected.put(Filter.class, 1);
+    expected.put(Aggregate.class, 1);
+    checkNodeTypeCount(sql, expected);
+  }
+
   /**
    * Matcher that succeeds for any collection that, when converted to strings
    * and sorted on those strings, matches the given reference string.

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 2314612..dd3861b 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -6593,7 +6593,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[10], ENAME=[$1], JOB=[$2], MGR=[null], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[7], SLACKER=[$8])
-  LogicalFilter(condition=[AND(=($7, 7), =($0, 10), IS NULL($3))])
+  LogicalFilter(condition=[AND(=($7, 7), IS NULL($3), =($0, 10))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>


[05/11] calcite git commit: [CALCITE-1682] New metadata providers for expression column origin and all predicates in plan

Posted by jc...@apache.org.
[CALCITE-1682] New metadata providers for expression column origin and all predicates in plan

Includes:
* RelNode type metadata provider
* Ranges containment-based simplification in conjunctive predicates


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

Branch: refs/heads/master
Commit: 41b05d784fd2e0ae81b09d013ef8a746036ca446
Parents: 478de56
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Mar 10 12:53:51 2017 +0000
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Apr 26 19:56:37 2017 +0100

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptCluster.java  |   1 +
 .../org/apache/calcite/plan/RelOptUtil.java     |  26 +-
 .../calcite/rel/metadata/BuiltInMetadata.java   |  57 +-
 .../metadata/DefaultRelMetadataProvider.java    |   3 +
 .../rel/metadata/RelMdAllPredicates.java        | 249 +++++++
 .../rel/metadata/RelMdExpressionLineage.java    | 450 +++++++++++++
 .../calcite/rel/metadata/RelMdNodeTypes.java    | 154 +++++
 .../apache/calcite/rel/metadata/RelMdUtil.java  |   1 +
 .../calcite/rel/metadata/RelMetadataQuery.java  |  60 ++
 .../calcite/rel/metadata/RelTableRef.java       |  62 ++
 .../org/apache/calcite/rex/LogicVisitor.java    |   4 +
 .../org/apache/calcite/rex/RexBiVisitor.java    |   2 +
 .../java/org/apache/calcite/rex/RexShuttle.java |   4 +
 .../org/apache/calcite/rex/RexSimplify.java     | 274 +++++++-
 .../apache/calcite/rex/RexTableInputRef.java    |  82 +++
 .../java/org/apache/calcite/rex/RexUtil.java    |  82 +++
 .../java/org/apache/calcite/rex/RexVisitor.java |   2 +
 .../org/apache/calcite/rex/RexVisitorImpl.java  |   4 +
 .../java/org/apache/calcite/sql/SqlKind.java    |   7 +
 .../org/apache/calcite/util/BuiltInMethod.java  |   6 +
 .../apache/calcite/test/RelMetadataTest.java    | 647 +++++++++++++++++++
 .../org/apache/calcite/test/RelOptRulesTest.xml |   2 +-
 22 files changed, 2151 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 f88e232..034d978 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
@@ -37,6 +37,7 @@ import java.util.concurrent.atomic.AtomicInteger;
  * optimization of a query.
  */
 public class RelOptCluster {
+
   //~ Instance fields --------------------------------------------------------
 
   private final RelDataTypeFactory typeFactory;

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 9af225e..662d316 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -118,10 +118,12 @@ import java.io.StringWriter;
 import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.BitSet;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -224,20 +226,32 @@ public abstract class RelOptUtil {
    * Returns a list of all tables used by this expression or its children
    */
   public static List<RelOptTable> findAllTables(RelNode rel) {
+    final Multimap<Class<? extends RelNode>, RelNode> nodes =
+        RelMetadataQuery.instance().getNodeTypes(rel);
     final List<RelOptTable> usedTables = new ArrayList<>();
-    new RelVisitor() {
-      @Override public void visit(RelNode node, int ordinal, RelNode parent) {
-        if (node instanceof TableScan) {
+    for (Entry<Class<? extends RelNode>, Collection<RelNode>> e : nodes.asMap().entrySet()) {
+      if (TableScan.class.isAssignableFrom(e.getKey())) {
+        for (RelNode node : e.getValue()) {
           usedTables.add(node.getTable());
         }
-        super.visit(node, ordinal, parent);
       }
-      // CHECKSTYLE: IGNORE 1
-    }.go(rel);
+    }
     return usedTables;
   }
 
   /**
+   * Returns a list of all tables used by this expression or its children
+   */
+  public static List<String> findAllTableQualifiedNames(RelNode rel) {
+    return Lists.transform(findAllTables(rel),
+        new Function<RelOptTable, String>() {
+          @Override public String apply(RelOptTable arg0) {
+            return arg0.getQualifiedName().toString();
+          }
+        });
+  }
+
+  /**
    * Returns a list of variables set by a relational expression or its
    * descendants.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 eb1ff79..0e0cbca 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
@@ -27,6 +27,7 @@ import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Multimap;
 
 import java.util.List;
 import java.util.Set;
@@ -160,6 +161,22 @@ public abstract class BuiltInMetadata {
     }
   }
 
+  /** Metadata about the node types and count in a relational expression. */
+  public interface NodeTypes extends Metadata {
+    MetadataDef<NodeTypes> DEF = MetadataDef.of(NodeTypes.class,
+        NodeTypes.Handler.class, BuiltInMethod.NODE_TYPES.method);
+
+    /**
+     *
+     */
+    Multimap<Class<? extends RelNode>, RelNode> getNodeTypes();
+
+    /** Handler API. */
+    interface Handler extends MetadataHandler<NodeTypes> {
+      Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(RelNode r, RelMetadataQuery mq);
+    }
+  }
+
   /** Metadata about the number of rows returned by a relational expression. */
   public interface RowCount extends Metadata {
     MetadataDef<RowCount> DEF = MetadataDef.of(RowCount.class,
@@ -370,6 +387,23 @@ public abstract class BuiltInMetadata {
     }
   }
 
+  /** Metadata about the origins of expressions. */
+  public interface ExpressionLineage extends Metadata {
+    MetadataDef<ExpressionLineage> DEF = MetadataDef.of(ExpressionLineage.class,
+        ExpressionLineage.Handler.class, BuiltInMethod.EXPRESSION_LINEAGE.method);
+
+    /**
+     *
+     */
+    Set<RexNode> getExpressionLineage(RexNode expression);
+
+    /** Handler API. */
+    interface Handler extends MetadataHandler<ExpressionLineage> {
+      Set<RexNode> getExpressionLineage(RelNode r, RelMetadataQuery mq,
+          RexNode expression);
+    }
+  }
+
   /** Metadata about the cost of evaluating a relational expression, including
    * all of its inputs. */
   public interface CumulativeCost extends Metadata {
@@ -461,6 +495,26 @@ public abstract class BuiltInMetadata {
     }
   }
 
+  /** Metadata about the predicates that hold in the rows emitted from a
+   * relational expression. */
+  public interface AllPredicates extends Metadata {
+    MetadataDef<AllPredicates> DEF = MetadataDef.of(AllPredicates.class,
+            AllPredicates.Handler.class, BuiltInMethod.ALL_PREDICATES.method);
+
+    /**
+     * Derives the predicates that hold on rows emitted from a relational
+     * expression.
+     *
+     * @return Predicate list
+     */
+    RelOptPredicateList getAllPredicates();
+
+    /** Handler API. */
+    interface Handler extends MetadataHandler<AllPredicates> {
+      RelOptPredicateList getAllPredicates(RelNode r, RelMetadataQuery mq);
+    }
+  }
+
   /** Metadata about the degree of parallelism of a relational expression, and
    * how its operators are assigned to processes with independent resource
    * pools. */
@@ -547,7 +601,8 @@ public abstract class BuiltInMetadata {
   /** The built-in forms of metadata. */
   interface All extends Selectivity, UniqueKeys, RowCount, DistinctRowCount,
       PercentageOriginalRows, ColumnUniqueness, ColumnOrigin, Predicates,
-      Collation, Distribution, Size, Parallelism, Memory {
+      Collation, Distribution, Size, Parallelism, Memory, AllPredicates,
+      ExpressionLineage, NodeTypes {
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
index bc04c4e..cb86698 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
@@ -43,6 +43,8 @@ public class DefaultRelMetadataProvider extends ChainedRelMetadataProvider {
         ImmutableList.of(
             RelMdPercentageOriginalRows.SOURCE,
             RelMdColumnOrigins.SOURCE,
+            RelMdExpressionLineage.SOURCE,
+            RelMdNodeTypes.SOURCE,
             RelMdRowCount.SOURCE,
             RelMdMaxRowCount.SOURCE,
             RelMdMinRowCount.SOURCE,
@@ -57,6 +59,7 @@ public class DefaultRelMetadataProvider extends ChainedRelMetadataProvider {
             RelMdSelectivity.SOURCE,
             RelMdExplainVisibility.SOURCE,
             RelMdPredicates.SOURCE,
+            RelMdAllPredicates.SOURCE,
             RelMdCollation.SOURCE));
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/core/src/main/java/org/apache/calcite/rel/metadata/RelMdAllPredicates.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdAllPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdAllPredicates.java
new file mode 100644
index 0000000..5f87a05
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdAllPredicates.java
@@ -0,0 +1,249 @@
+/*
+ * 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.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;
+import org.apache.calcite.rel.core.Exchange;
+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.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Utility to extract Predicates that are present in the (sub)plan
+ * starting at this node.
+ *
+ * This should be used to infer whether same filters are applied on
+ * a given plan by materialized view rewriting rules.
+ *
+ * The output predicates might contain references to columns produced
+ * by TableScan operators ({@link RexTableInputRef}). In turn, each TableScan
+ * operator is identified uniquely by its qualified name and an identifier.
+ *
+ * If the provider cannot infer the lineage for any of the expressions
+ * contain in any of the predicates, it will return null. Observe that
+ * this is different from the empty list of predicates, which means that
+ * there are not predicates in the (sub)plan.
+ *
+ */
+public class RelMdAllPredicates
+    implements MetadataHandler<BuiltInMetadata.AllPredicates> {
+  public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider
+      .reflectiveSource(BuiltInMethod.ALL_PREDICATES.method, new RelMdAllPredicates());
+
+  public MetadataDef<BuiltInMetadata.AllPredicates> getDef() {
+    return BuiltInMetadata.AllPredicates.DEF;
+  }
+
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.AllPredicates#getAllPredicates()},
+   * invoked using reflection.
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#getAllPredicates(RelNode)
+   */
+  public RelOptPredicateList getAllPredicates(RelNode rel, RelMetadataQuery mq) {
+    return null;
+  }
+
+  public RelOptPredicateList getAllPredicates(HepRelVertex rel, RelMetadataQuery mq) {
+    return mq.getAllPredicates(rel.getCurrentRel());
+  }
+
+  public RelOptPredicateList getAllPredicates(RelSubset rel,
+      RelMetadataQuery mq) {
+    return mq.getAllPredicates(Util.first(rel.getBest(), rel.getOriginal()));
+  }
+
+  /**
+   * Extract predicates for a table scan.
+   */
+  public RelOptPredicateList getAllPredicates(TableScan table, RelMetadataQuery mq) {
+    return RelOptPredicateList.EMPTY;
+  }
+
+  /**
+   * Extract predicates for a project.
+   */
+  public RelOptPredicateList getAllPredicates(Project project, RelMetadataQuery mq) {
+    return mq.getAllPredicates(project.getInput());
+  }
+
+  /**
+   * Add the Filter condition to the list obtained from the input.
+   */
+  public RelOptPredicateList getAllPredicates(Filter filter, RelMetadataQuery mq) {
+    final RelNode input = filter.getInput();
+    final RexBuilder rexBuilder = filter.getCluster().getRexBuilder();
+    final RexNode pred = filter.getCondition();
+
+    final RelOptPredicateList predsBelow = mq.getAllPredicates(input);
+    if (predsBelow == null) {
+      // Safety check
+      return null;
+    }
+
+    // Extract input fields referenced by Filter condition
+    final Set<RelDataTypeField> inputExtraFields = new LinkedHashSet<>();
+    final RelOptUtil.InputFinder inputFinder = new RelOptUtil.InputFinder(inputExtraFields);
+    pred.accept(inputFinder);
+    final ImmutableBitSet inputFieldsUsed = inputFinder.inputBitSet.build();
+
+    // Infer column origin expressions for given references
+    final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
+    for (int idx : inputFieldsUsed) {
+      final RexInputRef ref = RexInputRef.of(idx, filter.getRowType().getFieldList());
+      final Set<RexNode> originalExprs = mq.getExpressionLineage(filter, ref);
+      if (originalExprs == null) {
+        // Bail out
+        return null;
+      }
+      mapping.put(ref, originalExprs);
+    }
+
+    // Replace with new expressions and return union of predicates
+    return predsBelow.union(rexBuilder,
+        RelOptPredicateList.of(rexBuilder,
+            RelMdExpressionLineage.createAllPossibleExpressions(rexBuilder, pred, mapping)));
+  }
+
+  /**
+   * Add the Join condition to the list obtained from the input.
+   */
+  public RelOptPredicateList getAllPredicates(Join join, RelMetadataQuery mq) {
+    if (join.getJoinType() != JoinRelType.INNER) {
+      // We cannot map origin of this expression.
+      return null;
+    }
+
+    final RexBuilder rexBuilder = join.getCluster().getRexBuilder();
+    final RexNode pred = join.getCondition();
+    final RelNode leftInput = join.getLeft();
+    final RelNode rightInput = join.getRight();
+    final int nLeftColumns = leftInput.getRowType().getFieldList().size();
+
+    RelOptPredicateList newPreds = RelOptPredicateList.EMPTY;
+    for (RelNode input : join.getInputs()) {
+      final RelOptPredicateList inputPreds = mq.getAllPredicates(input);
+      if (inputPreds == null) {
+        // Bail out
+        return null;
+      }
+      newPreds = newPreds.union(rexBuilder, inputPreds);
+    }
+
+    // Extract input fields referenced by Join condition
+    final Set<RelDataTypeField> inputExtraFields = new LinkedHashSet<>();
+    final RelOptUtil.InputFinder inputFinder = new RelOptUtil.InputFinder(inputExtraFields);
+    pred.accept(inputFinder);
+    final ImmutableBitSet inputFieldsUsed = inputFinder.inputBitSet.build();
+
+    // Infer column origin expressions for given references
+    final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
+    for (int idx : inputFieldsUsed) {
+      if (idx < nLeftColumns) {
+        final RexInputRef inputRef = RexInputRef.of(idx, leftInput.getRowType().getFieldList());
+        final Set<RexNode> originalExprs = mq.getExpressionLineage(leftInput, inputRef);
+        if (originalExprs == null) {
+          // Bail out
+          return null;
+        }
+        final RexInputRef ref = RexInputRef.of(idx, join.getRowType().getFieldList());
+        mapping.put(ref, originalExprs);
+      } else {
+        // Right input.
+        final RexInputRef inputRef = RexInputRef.of(idx - nLeftColumns,
+                rightInput.getRowType().getFieldList());
+        final Set<RexNode> originalExprs = mq.getExpressionLineage(rightInput, inputRef);
+        if (originalExprs == null) {
+          // Bail out
+          return null;
+        }
+        final RexInputRef ref = RexInputRef.of(idx, join.getRowType().getFieldList());
+        mapping.put(ref, originalExprs);
+      }
+    }
+
+    // Replace with new expressions and return union of predicates
+    return newPreds.union(rexBuilder,
+        RelOptPredicateList.of(rexBuilder,
+            RelMdExpressionLineage.createAllPossibleExpressions(rexBuilder, pred, mapping)));
+  }
+
+  /**
+   * Extract predicates for an Aggregate.
+   */
+  public RelOptPredicateList getAllPredicates(Aggregate agg, RelMetadataQuery mq) {
+    return mq.getAllPredicates(agg.getInput());
+  }
+
+  /**
+   * Extract predicates for a Union.
+   */
+  public RelOptPredicateList getAllPredicates(Union union, RelMetadataQuery mq) {
+    final RexBuilder rexBuilder = union.getCluster().getRexBuilder();
+
+    RelOptPredicateList newPreds = RelOptPredicateList.EMPTY;
+    for (RelNode input : union.getInputs()) {
+      final RelOptPredicateList inputPreds = mq.getAllPredicates(input);
+      if (inputPreds == null) {
+        // Bail out
+        return null;
+      }
+      newPreds = newPreds.union(rexBuilder, inputPreds);
+    }
+    return newPreds;
+  }
+
+  /**
+   * Extract predicates for a Sort.
+   */
+  public RelOptPredicateList getAllPredicates(Sort sort, RelMetadataQuery mq) {
+    return mq.getAllPredicates(sort.getInput());
+  }
+
+  /**
+   * Extract predicates for an Exchange.
+   */
+  public RelOptPredicateList getAllPredicates(Exchange exchange,
+      RelMetadataQuery mq) {
+    return mq.getAllPredicates(exchange.getInput());
+  }
+
+}
+
+// End RelMdAllPredicates.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
new file mode 100644
index 0000000..5f9e5ba
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
@@ -0,0 +1,450 @@
+/*
+ * 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.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;
+import org.apache.calcite.rel.core.Exchange;
+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.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Function;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * RelMdExpressionLineage supplies a default implementation of
+ * {@link RelMetadataQuery#getExpressionLineage} for the standard logical algebra.
+ *
+ * The goal of this provider is to infer the lineage for the given expression.
+ *
+ * The output expressions might contain references to columns produced by TableScan
+ * operators ({@link RexTableInputRef}). In turn, each TableScan operator is identified
+ * uniquely by its qualified name and an identifier contained in .
+ *
+ * If the lineage cannot be inferred, we return null.
+ */
+public class RelMdExpressionLineage
+    implements MetadataHandler<BuiltInMetadata.ExpressionLineage> {
+  public static final RelMetadataProvider SOURCE =
+      ReflectiveRelMetadataProvider.reflectiveSource(
+          BuiltInMethod.EXPRESSION_LINEAGE.method, new RelMdExpressionLineage());
+
+  //~ Constructors -----------------------------------------------------------
+
+  private RelMdExpressionLineage() {}
+
+  //~ Methods ----------------------------------------------------------------
+
+  public MetadataDef<BuiltInMetadata.ExpressionLineage> getDef() {
+    return BuiltInMetadata.ExpressionLineage.DEF;
+  }
+
+  // Catch-all rule when none of the others apply.
+  public Set<RexNode> getExpressionLineage(RelNode rel,
+      RelMetadataQuery mq, RexNode outputExpression) {
+    return null;
+  }
+
+  public Set<RexNode> getExpressionLineage(HepRelVertex rel, RelMetadataQuery mq,
+      RexNode outputExpression) {
+    return mq.getExpressionLineage(rel.getCurrentRel(), outputExpression);
+  }
+
+  public Set<RexNode> getExpressionLineage(RelSubset rel,
+      RelMetadataQuery mq, RexNode outputExpression) {
+    return mq.getExpressionLineage(Util.first(rel.getBest(), rel.getOriginal()),
+        outputExpression);
+  }
+
+  /**
+   * Expression lineage from TableScan.
+   *
+   * We extract the fields referenced by the expression and we express them
+   * using {@link RexTableInputRef}.
+   */
+  public Set<RexNode> getExpressionLineage(TableScan rel,
+      RelMetadataQuery mq, RexNode outputExpression) {
+    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+
+    // Extract input fields referenced by expression
+    final Set<RelDataTypeField> inputExtraFields = new LinkedHashSet<>();
+    final RelOptUtil.InputFinder inputFinder = new RelOptUtil.InputFinder(inputExtraFields);
+    outputExpression.accept(inputFinder);
+    final ImmutableBitSet inputFieldsUsed = inputFinder.inputBitSet.build();
+
+    // Infer column origin expressions for given references
+    final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
+    for (int idx : inputFieldsUsed) {
+      final RexNode inputRef = RexTableInputRef.of(
+          new RelTableRef(rel.getTable().getQualifiedName().toString(), 0),
+          RexInputRef.of(idx, rel.getRowType().getFieldList()));
+      final Set<RexNode> originalExprs = Sets.newHashSet(inputRef);
+      final RexInputRef ref = RexInputRef.of(idx, rel.getRowType().getFieldList());
+      mapping.put(ref, originalExprs);
+    }
+
+    // Return result
+    return createAllPossibleExpressions(rexBuilder, outputExpression, mapping);
+  }
+
+  /**
+   * Expression lineage from Aggregate.
+   *
+   * If the expression references grouping sets or aggregation function results,
+   * we cannot extract the lineage and we return null.
+   */
+  public Set<RexNode> getExpressionLineage(Aggregate rel,
+      RelMetadataQuery mq, RexNode outputExpression) {
+    final RelNode input = rel.getInput();
+    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+
+    // Extract input fields referenced by expression
+    final Set<RelDataTypeField> inputExtraFields = new LinkedHashSet<>();
+    final RelOptUtil.InputFinder inputFinder = new RelOptUtil.InputFinder(inputExtraFields);
+    outputExpression.accept(inputFinder);
+    final ImmutableBitSet inputFieldsUsed = inputFinder.inputBitSet.build();
+
+    for (int idx : inputFieldsUsed) {
+      if (idx >= rel.getGroupCount()) {
+        // We cannot map origin of this expression.
+        return null;
+      }
+    }
+
+    // Infer column origin expressions for given references
+    final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
+    for (int idx : inputFieldsUsed) {
+      final RexInputRef inputRef = RexInputRef.of(rel.getGroupSet().nth(idx),
+          input.getRowType().getFieldList());
+      final Set<RexNode> originalExprs = mq.getExpressionLineage(input, inputRef);
+      if (originalExprs == null) {
+        // Bail out
+        return null;
+      }
+      final RexInputRef ref = RexInputRef.of(idx, rel.getRowType().getFieldList());
+      mapping.put(ref, originalExprs);
+    }
+
+    // Return result
+    return createAllPossibleExpressions(rexBuilder, outputExpression, mapping);
+  }
+
+  /**
+   * Expression lineage from Join.
+   *
+   * We only extract the lineage for INNER joins.
+   */
+  public Set<RexNode> getExpressionLineage(Join rel, RelMetadataQuery mq,
+      RexNode outputExpression) {
+    if (rel.getJoinType() != JoinRelType.INNER) {
+      // We cannot map origin of this expression.
+      return null;
+    }
+
+    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+    final RelNode leftInput = rel.getLeft();
+    final RelNode rightInput = rel.getRight();
+    final int nLeftColumns = leftInput.getRowType().getFieldList().size();
+
+    // Infer column origin expressions for given references
+    final Multimap<String, RelTableRef> qualifiedNamesToRefs = HashMultimap.create();
+    final Map<RelTableRef, RelTableRef> currentTablesMapping = new HashMap<>();
+    final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
+    for (int idx = 0; idx < rel.getRowType().getFieldList().size(); idx++) {
+      if (idx < nLeftColumns) {
+        final RexInputRef inputRef = RexInputRef.of(idx, leftInput.getRowType().getFieldList());
+        final Set<RexNode> originalExprs = mq.getExpressionLineage(leftInput, inputRef);
+        if (originalExprs == null) {
+          // Bail out
+          return null;
+        }
+        // Gather table references, left input references remain unchanged
+        final Set<RelTableRef> tableRefs =
+            RexUtil.gatherTableReferences(Lists.newArrayList(originalExprs));
+        for (RelTableRef leftRef : tableRefs) {
+          qualifiedNamesToRefs.put(leftRef.getQualifiedName(), leftRef);
+        }
+        mapping.put(RexInputRef.of(idx, rel.getRowType().getFieldList()), originalExprs);
+      } else {
+        // Right input.
+        final RexInputRef inputRef = RexInputRef.of(idx - nLeftColumns,
+                rightInput.getRowType().getFieldList());
+        final Set<RexNode> originalExprs = mq.getExpressionLineage(rightInput, inputRef);
+        if (originalExprs == null) {
+          // Bail out
+          return null;
+        }
+        // Gather table references, right input references might need to be
+        // updated if there are table names clashes with left input
+        final Set<RelTableRef> tableRefs =
+            RexUtil.gatherTableReferences(Lists.newArrayList(originalExprs));
+        for (RelTableRef rightRef : tableRefs) {
+          int shift = 0;
+          Collection<RelTableRef> lRefs = qualifiedNamesToRefs.get(
+              rightRef.getQualifiedName());
+          if (lRefs != null) {
+            shift = lRefs.size();
+          }
+          currentTablesMapping.put(rightRef,
+              new RelTableRef(rightRef.getQualifiedName(), shift + rightRef.getIdentifier()));
+        }
+        final Set<RexNode> updatedExprs = Sets.newHashSet(
+            Iterables.transform(
+                originalExprs,
+                new Function<RexNode, RexNode>() {
+                  @Override public RexNode apply(RexNode e) {
+                    return RexUtil.swapTableReferences(rexBuilder, e, currentTablesMapping);
+                  }
+                }
+          ));
+        mapping.put(RexInputRef.of(idx, rel.getRowType().getFieldList()), updatedExprs);
+      }
+    }
+
+    // Return result
+    return createAllPossibleExpressions(rexBuilder, outputExpression, mapping);
+  }
+
+  /**
+   * Expression lineage from Union.
+   *
+   * For Union operator, we might be able to extract multiple origins for the
+   * references in the given expression.
+   */
+  public Set<RexNode> getExpressionLineage(Union rel, RelMetadataQuery mq,
+      RexNode outputExpression) {
+    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+
+    // Infer column origin expressions for given references
+    final Multimap<String, RelTableRef> qualifiedNamesToRefs = HashMultimap.create();
+    final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
+    for (RelNode input : rel.getInputs()) {
+      final Map<RelTableRef, RelTableRef> currentTablesMapping = new HashMap<>();
+      for (int idx = 0; idx < input.getRowType().getFieldList().size(); idx++) {
+        final RexInputRef inputRef = RexInputRef.of(idx, input.getRowType().getFieldList());
+        final Set<RexNode> originalExprs = mq.getExpressionLineage(input, inputRef);
+        if (originalExprs == null) {
+          // Bail out
+          return null;
+        }
+
+        final RexInputRef ref = RexInputRef.of(idx, rel.getRowType().getFieldList());
+        // Gather table references, references might need to be
+        // updated
+        final Set<RelTableRef> tableRefs =
+            RexUtil.gatherTableReferences(Lists.newArrayList(originalExprs));
+        for (RelTableRef tableRef : tableRefs) {
+          int shift = 0;
+          Collection<RelTableRef> lRefs = qualifiedNamesToRefs.get(
+              tableRef.getQualifiedName());
+          if (lRefs != null) {
+            shift = lRefs.size();
+          }
+          currentTablesMapping.put(tableRef,
+              new RelTableRef(tableRef.getQualifiedName(), shift + tableRef.getIdentifier()));
+        }
+        final Set<RexNode> updatedExprs = Sets.newHashSet(
+            Iterables.transform(
+                originalExprs,
+                new Function<RexNode, RexNode>() {
+                  @Override public RexNode apply(RexNode e) {
+                    return RexUtil.swapTableReferences(rexBuilder, e, currentTablesMapping);
+                  }
+                }
+          ));
+        final Set<RexNode> set = mapping.get(ref);
+        if (set != null) {
+          set.addAll(updatedExprs);
+        } else {
+          mapping.put(ref, updatedExprs);
+        }
+      }
+      // Add to existing qualified names
+      for (RelTableRef newRef : currentTablesMapping.values()) {
+        qualifiedNamesToRefs.put(newRef.getQualifiedName(), newRef);
+      }
+    }
+
+    // Return result
+    return createAllPossibleExpressions(rexBuilder, outputExpression, mapping);
+  }
+
+  /**
+   * Expression lineage from Project.
+   */
+  public Set<RexNode> getExpressionLineage(Project rel,
+      final RelMetadataQuery mq, RexNode outputExpression) {
+    final RelNode input = rel.getInput();
+    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+
+    // Extract input fields referenced by expression
+    final Set<RelDataTypeField> inputExtraFields = new LinkedHashSet<>();
+    final RelOptUtil.InputFinder inputFinder = new RelOptUtil.InputFinder(inputExtraFields);
+    outputExpression.accept(inputFinder);
+    final ImmutableBitSet inputFieldsUsed = inputFinder.inputBitSet.build();
+
+    // Infer column origin expressions for given references
+    final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
+    for (int idx : inputFieldsUsed) {
+      final RexNode inputExpr = rel.getChildExps().get(idx);
+      final Set<RexNode> originalExprs = mq.getExpressionLineage(input, inputExpr);
+      if (originalExprs == null) {
+        // Bail out
+        return null;
+      }
+      final RexInputRef ref = RexInputRef.of(idx, rel.getRowType().getFieldList());
+      mapping.put(ref, originalExprs);
+    }
+
+    // Return result
+    return createAllPossibleExpressions(rexBuilder, outputExpression, mapping);
+  }
+
+  /**
+   * Expression lineage from Filter.
+   */
+  public Set<RexNode> getExpressionLineage(Filter rel,
+      RelMetadataQuery mq, RexNode outputExpression) {
+    return mq.getExpressionLineage(rel.getInput(), outputExpression);
+  }
+
+  /**
+   * Expression lineage from Sort.
+   */
+  public Set<RexNode> getExpressionLineage(Sort rel, RelMetadataQuery mq,
+      RexNode outputExpression) {
+    return mq.getExpressionLineage(rel.getInput(), outputExpression);
+  }
+
+  /**
+   * Expression lineage from Exchange.
+   */
+  public Set<RexNode> getExpressionLineage(Exchange rel,
+      RelMetadataQuery mq, RexNode outputExpression) {
+    return mq.getExpressionLineage(rel.getInput(), outputExpression);
+  }
+
+  /**
+   * Given an expression, it will create all equivalent expressions resulting
+   * from replacing all possible combinations of references in the mapping by
+   * the corresponding expressions.
+   *
+   * @param rexBuilder rexBuilder
+   * @param expr expression
+   * @param mapping mapping
+   * @return set of resulting expressions equivalent to the input expression
+   */
+  protected static Set<RexNode> createAllPossibleExpressions(RexBuilder rexBuilder,
+      RexNode expr, Map<RexInputRef, Set<RexNode>> mapping) {
+    // Extract input fields referenced by expression
+    final Set<RelDataTypeField> inputExtraFields = new LinkedHashSet<>();
+    final RelOptUtil.InputFinder inputFinder = new RelOptUtil.InputFinder(inputExtraFields);
+    expr.accept(inputFinder);
+    final ImmutableBitSet predFieldsUsed = inputFinder.inputBitSet.build();
+
+    return createAllPossibleExpressions(rexBuilder, expr, predFieldsUsed, mapping,
+        new HashMap<RexInputRef, RexNode>());
+  }
+
+  private static Set<RexNode> createAllPossibleExpressions(RexBuilder rexBuilder,
+      RexNode expr, ImmutableBitSet predFieldsUsed, Map<RexInputRef, Set<RexNode>> mapping,
+      Map<RexInputRef, RexNode> singleMapping) {
+    final RexInputRef inputRef = mapping.keySet().iterator().next();
+    final Set<RexNode> replacements = mapping.remove(inputRef);
+    Set<RexNode> result = new HashSet<>();
+    assert !replacements.isEmpty();
+    if (predFieldsUsed.indexOf(inputRef.getIndex()) != -1) {
+      for (RexNode replacement : replacements) {
+        singleMapping.put(inputRef, replacement);
+        createExpressions(rexBuilder, expr, predFieldsUsed, mapping, singleMapping, result);
+        singleMapping.remove(inputRef);
+      }
+    } else {
+      createExpressions(rexBuilder, expr, predFieldsUsed, mapping, singleMapping, result);
+    }
+    mapping.put(inputRef, replacements);
+    return result;
+  }
+
+  private static void createExpressions(RexBuilder rexBuilder,
+      RexNode expr, ImmutableBitSet predFieldsUsed, Map<RexInputRef, Set<RexNode>> mapping,
+      Map<RexInputRef, RexNode> singleMapping, Set<RexNode> result) {
+    if (mapping.isEmpty()) {
+      final RexReplacer replacer = new RexReplacer(singleMapping);
+      final List<RexNode> updatedPreds = new ArrayList<>(
+          RelOptUtil.conjunctions(
+              rexBuilder.copy(expr)));
+      replacer.mutate(updatedPreds);
+      result.addAll(updatedPreds);
+    } else {
+      result.addAll(
+          createAllPossibleExpressions(
+              rexBuilder, expr, predFieldsUsed, mapping, singleMapping));
+    }
+  }
+
+  /**
+   * Replaces expressions with their equivalences. Note that we only have to
+   * look for RexInputRef.
+   */
+  private static class RexReplacer extends RexShuttle {
+    private final Map<RexInputRef, RexNode> replacementValues;
+
+    RexReplacer(Map<RexInputRef, RexNode> replacementValues) {
+      this.replacementValues = replacementValues;
+    }
+
+    @Override public RexNode visitInputRef(RexInputRef inputRef) {
+      return replacementValues.get(inputRef);
+    }
+  }
+
+}
+
+// End RelMdExpressionLineage.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/core/src/main/java/org/apache/calcite/rel/metadata/RelMdNodeTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdNodeTypes.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdNodeTypes.java
new file mode 100644
index 0000000..566df3a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdNodeTypes.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.rel.metadata;
+
+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;
+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.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+/**
+ * RelMdNodeTypeCount supplies a default implementation of
+ * {@link RelMetadataQuery#getNodeTypes} for the standard logical algebra.
+ */
+public class RelMdNodeTypes
+    implements MetadataHandler<BuiltInMetadata.NodeTypes> {
+  public static final RelMetadataProvider SOURCE =
+      ReflectiveRelMetadataProvider.reflectiveSource(
+          BuiltInMethod.NODE_TYPES.method, new RelMdNodeTypes());
+
+  //~ Methods ----------------------------------------------------------------
+
+  public MetadataDef<BuiltInMetadata.NodeTypes> getDef() {
+    return BuiltInMetadata.NodeTypes.DEF;
+  }
+
+  /** Catch-all implementation for
+   * {@link BuiltInMetadata.NodeTypeCount#getNodeTypes()},
+   * invoked using reflection.
+   *
+   * @see org.apache.calcite.rel.metadata.RelMetadataQuery#getNodeTypes(RelNode)
+   */
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(RelNode rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, RelNode.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(HepRelVertex rel,
+      RelMetadataQuery mq) {
+    return mq.getNodeTypes(rel.getCurrentRel());
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(RelSubset rel,
+      RelMetadataQuery mq) {
+    return mq.getNodeTypes(Util.first(rel.getBest(), rel.getOriginal()));
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Union rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Union.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Intersect rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Intersect.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Minus rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Minus.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Filter rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Filter.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Calc rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Calc.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Project rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Project.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Sort rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Sort.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Join rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Join.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(SemiJoin rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, SemiJoin.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Aggregate rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Aggregate.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(TableScan rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, TableScan.class, mq);
+  }
+
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(Values rel,
+      RelMetadataQuery mq) {
+    return getNodeTypes(rel, Values.class, mq);
+  }
+
+  private static Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(RelNode rel,
+      Class<? extends RelNode> c, RelMetadataQuery mq) {
+    final Multimap<Class<? extends RelNode>, RelNode> nodeTypeCount = ArrayListMultimap.create();
+    for (RelNode input : rel.getInputs()) {
+      Multimap<Class<? extends RelNode>, RelNode> partialNodeTypeCount =
+          mq.getNodeTypes(input);
+      if (partialNodeTypeCount == null) {
+        return null;
+      }
+      nodeTypeCount.putAll(partialNodeTypeCount);
+    }
+    nodeTypeCount.put(c, rel);
+    return nodeTypeCount;
+  }
+
+}
+
+// End RelMdNodeTypes.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 7b63ac9..4ef5592 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
@@ -843,6 +843,7 @@ public class RelMdUtil {
     }
     return alreadySorted && alreadySmaller;
   }
+
 }
 
 // End RelMdUtil.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 85f747c..9dd19dc 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
@@ -29,6 +29,7 @@ import org.apache.calcite.util.ImmutableBitSet;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
 
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
@@ -84,6 +85,7 @@ public class RelMetadataQuery {
 
   private BuiltInMetadata.Collation.Handler collationHandler;
   private BuiltInMetadata.ColumnOrigin.Handler columnOriginHandler;
+  private BuiltInMetadata.ExpressionLineage.Handler expressionLineageHandler;
   private BuiltInMetadata.ColumnUniqueness.Handler columnUniquenessHandler;
   private BuiltInMetadata.CumulativeCost.Handler cumulativeCostHandler;
   private BuiltInMetadata.DistinctRowCount.Handler distinctRowCountHandler;
@@ -97,6 +99,8 @@ public class RelMetadataQuery {
   private BuiltInMetadata.PercentageOriginalRows.Handler percentageOriginalRowsHandler;
   private BuiltInMetadata.PopulationSize.Handler populationSizeHandler;
   private BuiltInMetadata.Predicates.Handler predicatesHandler;
+  private BuiltInMetadata.AllPredicates.Handler allPredicatesHandler;
+  private BuiltInMetadata.NodeTypes.Handler nodeTypesHandler;
   private BuiltInMetadata.RowCount.Handler rowCountHandler;
   private BuiltInMetadata.Selectivity.Handler selectivityHandler;
   private BuiltInMetadata.Size.Handler sizeHandler;
@@ -114,6 +118,7 @@ public class RelMetadataQuery {
     this.metadataProvider = Preconditions.checkNotNull(metadataProvider);
     this.collationHandler = prototype.collationHandler;
     this.columnOriginHandler = prototype.columnOriginHandler;
+    this.expressionLineageHandler = prototype.expressionLineageHandler;
     this.columnUniquenessHandler = prototype.columnUniquenessHandler;
     this.cumulativeCostHandler = prototype.cumulativeCostHandler;
     this.distinctRowCountHandler = prototype.distinctRowCountHandler;
@@ -127,6 +132,8 @@ public class RelMetadataQuery {
     this.percentageOriginalRowsHandler = prototype.percentageOriginalRowsHandler;
     this.populationSizeHandler = prototype.populationSizeHandler;
     this.predicatesHandler = prototype.predicatesHandler;
+    this.allPredicatesHandler = prototype.allPredicatesHandler;
+    this.nodeTypesHandler = prototype.nodeTypesHandler;
     this.rowCountHandler = prototype.rowCountHandler;
     this.selectivityHandler = prototype.selectivityHandler;
     this.sizeHandler = prototype.sizeHandler;
@@ -162,6 +169,7 @@ public class RelMetadataQuery {
     this.metadataProvider = null;
     this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
     this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
+    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
     this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
     this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
     this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
@@ -176,6 +184,8 @@ public class RelMetadataQuery {
         initialHandler(BuiltInMetadata.PercentageOriginalRows.Handler.class);
     this.populationSizeHandler = initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
     this.predicatesHandler = initialHandler(BuiltInMetadata.Predicates.Handler.class);
+    this.allPredicatesHandler = initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
+    this.nodeTypesHandler = initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
     this.rowCountHandler = initialHandler(BuiltInMetadata.RowCount.Handler.class);
     this.selectivityHandler = initialHandler(BuiltInMetadata.Selectivity.Handler.class);
     this.sizeHandler = initialHandler(BuiltInMetadata.Size.Handler.class);
@@ -191,6 +201,24 @@ public class RelMetadataQuery {
 
   /**
    * Returns the
+   * {@link BuiltInMetadata.NodeTypeCount#getNodeTypeCount()}
+   * statistic.
+   *
+   * @param rel the relational expression
+   * @return
+   */
+  public Multimap<Class<? extends RelNode>, RelNode> getNodeTypes(RelNode rel) {
+    for (;;) {
+      try {
+        return nodeTypesHandler.getNodeTypes(rel, this);
+      } catch (JaninoRelMetadataProvider.NoHandler e) {
+        nodeTypesHandler = revise(e.relClass, BuiltInMetadata.NodeTypes.DEF);
+      }
+    }
+  }
+
+  /**
+   * Returns the
    * {@link BuiltInMetadata.RowCount#getRowCount()}
    * statistic.
    *
@@ -352,6 +380,20 @@ public class RelMetadataQuery {
   }
 
   /**
+   * Determines the origin of a column.
+   */
+  public Set<RexNode> getExpressionLineage(RelNode rel, RexNode expression) {
+    for (;;) {
+      try {
+        return expressionLineageHandler.getExpressionLineage(rel, this, expression);
+      } catch (JaninoRelMetadataProvider.NoHandler e) {
+        expressionLineageHandler =
+            revise(e.relClass, BuiltInMetadata.ExpressionLineage.DEF);
+      }
+    }
+  }
+
+  /**
    * Determines the origin of a {@link RelNode}, provided it maps to a single
    * table, optionally with filtering and projection.
    *
@@ -749,6 +791,24 @@ public class RelMetadataQuery {
 
   /**
    * Returns the
+   * {@link BuiltInMetadata.AllPredicates#getAllPredicates()}
+   * statistic.
+   *
+   * @param rel the relational expression
+   * @return All predicates within and below this RelNode
+   */
+  public RelOptPredicateList getAllPredicates(RelNode rel) {
+    for (;;) {
+      try {
+        return allPredicatesHandler.getAllPredicates(rel, this);
+      } catch (JaninoRelMetadataProvider.NoHandler e) {
+        allPredicatesHandler = revise(e.relClass, BuiltInMetadata.AllPredicates.DEF);
+      }
+    }
+  }
+
+  /**
+   * Returns the
    * {@link BuiltInMetadata.ExplainVisibility#isVisibleInExplain(SqlExplainLevel)}
    * statistic.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/core/src/main/java/org/apache/calcite/rel/metadata/RelTableRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelTableRef.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelTableRef.java
new file mode 100644
index 0000000..bdd032f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelTableRef.java
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+/**
+ *
+ *
+ */
+public class RelTableRef {
+
+  private final String qualifiedName;
+  private final int identifier;
+  private final String digest;
+
+  public RelTableRef(String qualifiedName, int identifier) {
+    this.qualifiedName = qualifiedName;
+    this.identifier = identifier;
+    this.digest = qualifiedName + ".#" + identifier;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public boolean equals(Object obj) {
+    return this == obj
+        || obj instanceof RelTableRef
+        && qualifiedName.equals(((RelTableRef) obj).qualifiedName)
+        && identifier == ((RelTableRef) obj).identifier;
+  }
+
+  @Override public int hashCode() {
+    return digest.hashCode();
+  }
+
+  public String getQualifiedName() {
+    return qualifiedName;
+  }
+
+  public int getIdentifier() {
+    return identifier;
+  }
+
+  @Override public String toString() {
+    return digest;
+  }
+
+}
+
+// End RelTableRef.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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
index a16a834..f632e94 100644
--- a/core/src/main/java/org/apache/calcite/rex/LogicVisitor.java
+++ b/core/src/main/java/org/apache/calcite/rex/LogicVisitor.java
@@ -163,6 +163,10 @@ public class LogicVisitor implements RexBiVisitor<Logic, Logic> {
     return end(subQuery, arg);
   }
 
+  @Override public Logic visitTableInputRef(RexTableInputRef ref, Logic arg) {
+    return end(ref, arg);
+  }
+
   @Override public Logic visitPatternFieldRef(RexPatternFieldRef ref, Logic arg) {
     return end(ref, arg);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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
index 8e0b014..aa494c2 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBiVisitor.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBiVisitor.java
@@ -48,6 +48,8 @@ public interface RexBiVisitor<R, P> {
 
   R visitSubQuery(RexSubQuery subQuery, P arg);
 
+  R visitTableInputRef(RexTableInputRef ref, P arg);
+
   R visitPatternFieldRef(RexPatternFieldRef ref, P arg);
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 b642503..6714be1 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
@@ -89,6 +89,10 @@ public class RexShuttle implements RexVisitor<RexNode> {
     }
   }
 
+  @Override public RexNode visitTableInputRef(RexTableInputRef ref) {
+    return ref;
+  }
+
   @Override public RexNode visitPatternFieldRef(RexPatternFieldRef fieldRef) {
     return fieldRef;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index a910778..fb791f5 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -29,9 +29,11 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.BoundType;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -563,8 +565,8 @@ public class RexSimplify {
       return simplify(terms.get(0));
     }
     // Try to simplify the expression
-    final Multimap<String, Pair<String, RexNode>> equalityTerms =
-        ArrayListMultimap.create();
+    final Multimap<String, Pair<String, RexNode>> equalityTerms = ArrayListMultimap.create();
+    final Map<String, Pair<Range, List<RexNode>>> rangeTerms = new HashMap<>();
     final Map<String, String> equalityConstantTerms = new HashMap<>();
     final Set<String> negatedTerms = new HashSet<>();
     final Set<String> nullOperands = new HashSet<>();
@@ -611,23 +613,23 @@ public class RexSimplify {
           RexCall rightCast = (RexCall) right;
           comparedOperands.add(rightCast.getOperands().get(0).toString());
         }
-        // Check for equality on different constants. If the same ref or
-        // CAST(ref) is equal to different constants, this condition cannot be
-        // satisfied, and hence it can be evaluated to FALSE.
+        // Check for equality on different constants. If the same ref or CAST(ref)
+        // is equal to different constants, this condition cannot be satisfied,
+        // and hence it can be evaluated to FALSE
+        final boolean leftRef = RexUtil.isReferenceOrAccess(left, true);
+        final boolean rightRef = RexUtil.isReferenceOrAccess(right, true);
+        final boolean leftConstant = left.isA(SqlKind.LITERAL);
+        final boolean rightConstant = right.isA(SqlKind.LITERAL);
         if (term.getKind() == SqlKind.EQUALS) {
-          final boolean leftRef = RexUtil.isReferenceOrAccess(left, true);
-          final boolean rightRef = RexUtil.isReferenceOrAccess(right, true);
-          if (right instanceof RexLiteral && leftRef) {
+          if (leftRef && rightConstant) {
             final String literal = right.toString();
-            final String prevLiteral =
-                equalityConstantTerms.put(left.toString(), literal);
+            final String prevLiteral = equalityConstantTerms.put(left.toString(), literal);
             if (prevLiteral != null && !literal.equals(prevLiteral)) {
               return rexBuilder.makeLiteral(false);
             }
-          } else if (left instanceof RexLiteral && rightRef) {
+          } else if (leftConstant && rightRef) {
             final String literal = left.toString();
-            final String prevLiteral =
-                equalityConstantTerms.put(right.toString(), literal);
+            final String prevLiteral = equalityConstantTerms.put(right.toString(), literal);
             if (prevLiteral != null && !literal.equals(prevLiteral)) {
               return rexBuilder.makeLiteral(false);
             }
@@ -637,19 +639,39 @@ public class RexSimplify {
         }
         // Assume the expression a > 5 is part of a Filter condition.
         // Then we can derive the negated term: a <= 5.
-        // But as comparison is string-based and thus operands order-dependent,
+        // But as the comparison is string based and thus operands order dependent,
         // we should also add the inverted negated term: 5 >= a.
-        // Observe that for creating the inverted term we invert the list of
-        // operands.
+        // Observe that for creating the inverted term we invert the list of operands.
         RexNode negatedTerm = RexUtil.negate(rexBuilder, call);
         if (negatedTerm != null) {
           negatedTerms.add(negatedTerm.toString());
-          RexNode invertNegatedTerm =
-              RexUtil.invert(rexBuilder, (RexCall) negatedTerm);
+          RexNode invertNegatedTerm = RexUtil.invert(rexBuilder, (RexCall) negatedTerm);
           if (invertNegatedTerm != null) {
             negatedTerms.add(invertNegatedTerm.toString());
           }
         }
+        // Range
+        SqlKind comparison = null;
+        RexNode ref = null;
+        RexLiteral constant = null;
+        if (leftRef && rightConstant) {
+          comparison = term.getKind();
+          ref = left;
+          constant = (RexLiteral) right;
+        } else if (leftConstant && rightRef) {
+          comparison = term.getKind().reverse();
+          constant = (RexLiteral) left;
+          ref = right;
+        }
+        if (comparison != null
+            && comparison != SqlKind.NOT_EQUALS) { // NOT_EQUALS not supported
+          final RexNode result = processRange(rexBuilder, terms, rangeTerms,
+                  term, ref, constant, comparison);
+          if (result != null) {
+            // Not satisfiable
+            return result;
+          }
+        }
         break;
       case IN:
         comparedOperands.add(((RexCall) term).operands.get(0).toString());
@@ -717,8 +739,7 @@ public class RexSimplify {
       if (!RexUtil.isDeterministic(notDisjunction)) {
         continue;
       }
-      final List<String> terms2Set =
-          RexUtil.strings(RelOptUtil.conjunctions(notDisjunction));
+      final List<String> terms2Set = RexUtil.strings(RelOptUtil.conjunctions(notDisjunction));
       if (termsSet.containsAll(terms2Set)) {
         return rexBuilder.makeLiteral(false);
       }
@@ -800,6 +821,219 @@ public class RexSimplify {
     }
   }
 
+  private static RexNode processRange(RexBuilder rexBuilder,
+      List<RexNode> terms, Map<String, Pair<Range, List<RexNode>>> rangeTerms,
+      RexNode term, RexNode ref, RexLiteral constant, SqlKind comparison) {
+    final Comparable v0 = constant.getValue();
+    Pair<Range, List<RexNode>> p = rangeTerms.get(ref.toString());
+    if (p == null) {
+      Range r;
+      switch (comparison) {
+      case EQUALS:
+        r = Range.singleton(v0);
+        break;
+      case LESS_THAN:
+        r = Range.lessThan(v0);
+        break;
+      case LESS_THAN_OR_EQUAL:
+        r = Range.atMost(v0);
+        break;
+      case GREATER_THAN:
+        r = Range.greaterThan(v0);
+        break;
+      case GREATER_THAN_OR_EQUAL:
+        r = Range.atLeast(v0);
+        break;
+      default:
+        throw new AssertionError();
+      }
+      rangeTerms.put(ref.toString(),
+              new Pair(r, ImmutableList.of(term)));
+    } else {
+      // Exists
+      boolean removeUpperBound = false;
+      boolean removeLowerBound = false;
+      Range r = p.left;
+      switch (comparison) {
+      case EQUALS:
+        if (!r.contains(v0)) {
+          // Range is empty, not satisfiable
+          return rexBuilder.makeLiteral(false);
+        }
+        rangeTerms.put(ref.toString(),
+                new Pair(Range.singleton(v0), ImmutableList.of(term)));
+        // remove
+        terms.removeAll(p.right);
+        break;
+      case LESS_THAN: {
+        int comparisonResult = 0;
+        if (r.hasUpperBound()) {
+          comparisonResult = v0.compareTo(r.upperEndpoint());
+        }
+        if (comparisonResult <= 0) {
+          // 1) No upper bound, or
+          // 2) We need to open the upper bound, or
+          // 3) New upper bound is lower than old upper bound
+          if (r.hasLowerBound()) {
+            if (v0.compareTo(r.lowerEndpoint()) < 0) {
+              // Range is empty, not satisfiable
+              return rexBuilder.makeLiteral(false);
+            }
+            // a <= x < b OR a < x < b
+            r = Range.range(r.lowerEndpoint(), r.lowerBoundType(),
+                    v0, BoundType.OPEN);
+          } else {
+            // x < b
+            r = Range.lessThan(v0);
+          }
+
+          if (r.isEmpty()) {
+            // Range is empty, not satisfiable
+            return rexBuilder.makeLiteral(false);
+          }
+
+          // remove prev upper bound
+          removeUpperBound = true;
+        } else {
+          // Remove this term as it is contained in current upper bound
+          terms.remove(term);
+        }
+        break;
+      }
+      case LESS_THAN_OR_EQUAL: {
+        int comparisonResult = -1;
+        if (r.hasUpperBound()) {
+          comparisonResult = v0.compareTo(r.upperEndpoint());
+        }
+        if (comparisonResult < 0) {
+          // 1) No upper bound, or
+          // 2) New upper bound is lower than old upper bound
+          if (r.hasLowerBound()) {
+            if (v0.compareTo(r.lowerEndpoint()) < 0) {
+              // Range is empty, not satisfiable
+              return rexBuilder.makeLiteral(false);
+            }
+            // a <= x <= b OR a < x <= b
+            r = Range.range(r.lowerEndpoint(), r.lowerBoundType(),
+                    v0, BoundType.CLOSED);
+          } else {
+            // x <= b
+            r = Range.atMost(v0);
+          }
+
+          if (r.isEmpty()) {
+            // Range is empty, not satisfiable
+            return rexBuilder.makeLiteral(false);
+          }
+
+          // remove prev upper bound
+          removeUpperBound = true;
+        } else {
+          // Remove this term as it is contained in current upper bound
+          terms.remove(term);
+        }
+        break;
+      }
+      case GREATER_THAN: {
+        int comparisonResult = 0;
+        if (r.hasLowerBound()) {
+          comparisonResult = v0.compareTo(r.lowerEndpoint());
+        }
+        if (comparisonResult >= 0) {
+          // 1) No lower bound, or
+          // 2) We need to open the lower bound, or
+          // 3) New lower bound is greater than old lower bound
+          if (r.hasUpperBound()) {
+            if (v0.compareTo(r.upperEndpoint()) > 0) {
+              // Range is empty, not satisfiable
+              return rexBuilder.makeLiteral(false);
+            }
+            // a < x <= b OR a < x < b
+            r = Range.range(v0, BoundType.OPEN,
+                    r.upperEndpoint(), r.upperBoundType());
+          } else {
+            // x > a
+            r = Range.greaterThan(v0);
+          }
+
+          if (r.isEmpty()) {
+            // Range is empty, not satisfiable
+            return rexBuilder.makeLiteral(false);
+          }
+
+          // remove prev lower bound
+          removeLowerBound = true;
+        } else {
+          // Remove this term as it is contained in current lower bound
+          terms.remove(term);
+        }
+        break;
+      }
+      case GREATER_THAN_OR_EQUAL: {
+        int comparisonResult = 1;
+        if (r.hasLowerBound()) {
+          comparisonResult = v0.compareTo(r.lowerEndpoint());
+        }
+        if (comparisonResult > 0) {
+          // 1) No lower bound, or
+          // 2) New lower bound is greater than old lower bound
+          if (r.hasUpperBound()) {
+            if (v0.compareTo(r.upperEndpoint()) > 0) {
+              // Range is empty, not satisfiable
+              return rexBuilder.makeLiteral(false);
+            }
+            // a <= x <= b OR a <= x < b
+            r = Range.range(v0, BoundType.CLOSED,
+                    r.upperEndpoint(), r.upperBoundType());
+          } else {
+            // x >= a
+            r = Range.atLeast(v0);
+          }
+
+          if (r.isEmpty()) {
+            // Range is empty, not satisfiable
+            return rexBuilder.makeLiteral(false);
+          }
+
+          // remove prev lower bound
+          removeLowerBound = true;
+        } else {
+          // Remove this term as it is contained in current lower bound
+          terms.remove(term);
+        }
+        break;
+      }
+      default:
+        throw new AssertionError();
+      }
+      if (removeUpperBound) {
+        ImmutableList.Builder<RexNode> newBounds = ImmutableList.builder();
+        for (RexNode e : p.right) {
+          if (e.isA(SqlKind.LESS_THAN) || e.isA(SqlKind.LESS_THAN_OR_EQUAL)) {
+            terms.remove(e);
+          } else {
+            newBounds.add(e);
+          }
+        }
+        newBounds.add(term);
+        rangeTerms.put(ref.toString(), new Pair(r, newBounds.build()));
+      } else if (removeLowerBound) {
+        ImmutableList.Builder<RexNode> newBounds = ImmutableList.builder();
+        for (RexNode e : p.right) {
+          if (e.isA(SqlKind.GREATER_THAN) || e.isA(SqlKind.GREATER_THAN_OR_EQUAL)) {
+            terms.remove(e);
+          } else {
+            newBounds.add(e);
+          }
+        }
+        newBounds.add(term);
+        rangeTerms.put(ref.toString(), new Pair(r, newBounds.build()));
+      }
+    }
+    // Default
+    return null;
+  }
+
 }
 
 // End RexSimplify.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
new file mode 100644
index 0000000..7e2d26b
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
@@ -0,0 +1,82 @@
+/*
+ * 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.rel.metadata.RelTableRef;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
+
+/**
+ * Variable which references a field of an input relational expression
+ */
+public class RexTableInputRef extends RexInputRef {
+
+  private final RelTableRef tableRef;
+
+  public RexTableInputRef(RelTableRef tableRef, int index, RelDataType type) {
+    super(index, type);
+    this.tableRef = tableRef;
+    this.digest = tableRef.toString() + ".$" + index;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public boolean equals(Object obj) {
+    return this == obj
+        || obj instanceof RexTableInputRef
+        && tableRef.equals(((RexTableInputRef) obj).tableRef)
+        && index == ((RexTableInputRef) obj).index;
+  }
+
+  @Override public int hashCode() {
+    return digest.hashCode();
+  }
+
+  public RelTableRef getTableRef() {
+    return tableRef;
+  }
+
+  public String getQualifiedName() {
+    return tableRef.getQualifiedName();
+  }
+
+  public int getIdentifier() {
+    return tableRef.getIdentifier();
+  }
+
+  public static RexTableInputRef of(RelTableRef tableRef, int index, RelDataType type) {
+    return new RexTableInputRef(tableRef, index, type);
+  }
+
+  public static RexTableInputRef of(RelTableRef tableRef, RexInputRef ref) {
+    return new RexTableInputRef(tableRef, ref.getIndex(), ref.getType());
+  }
+
+  @Override public <R> R accept(RexVisitor<R> visitor) {
+    return visitor.visitTableInputRef(this);
+  }
+
+  @Override public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitTableInputRef(this, arg);
+  }
+
+  @Override public SqlKind getKind() {
+    return SqlKind.TABLE_INPUT_REF;
+  }
+}
+
+// End RexTableInputRef.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 f174f41..0af3fc7 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -24,6 +24,7 @@ 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.metadata.RelTableRef;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
@@ -490,6 +491,10 @@ public class RexUtil {
       return false;
     }
 
+    @Override public Boolean visitTableInputRef(RexTableInputRef ref) {
+      return false;
+    }
+
     @Override public Boolean visitPatternFieldRef(RexPatternFieldRef fieldRef) {
       return false;
     }
@@ -821,6 +826,28 @@ public class RexUtil {
     return false;
   }
 
+  /**
+   * Returns whether a given tree contains any {link RexTableInputRef} nodes.
+   *
+   * @param node a RexNode tree
+   * @return first such node found or null if it there is no such node
+   */
+  public static RexTableInputRef containsTableInputRef(RexNode node) {
+    try {
+      RexVisitor<Void> visitor =
+          new RexVisitorImpl<Void>(true) {
+            public Void visitTableInputRef(RexTableInputRef inputRef) {
+              throw new Util.FoundOne(inputRef);
+            }
+          };
+      node.accept(visitor);
+      return null;
+    } catch (Util.FoundOne e) {
+      Util.swallow(e, null);
+      return (RexTableInputRef) e.getNode();
+    }
+  }
+
   public static boolean isAtomic(RexNode expr) {
     return (expr instanceof RexLiteral) || (expr instanceof RexVariable);
   }
@@ -1840,6 +1867,61 @@ public class RexUtil {
     }
   }
 
+  public static RexNode swapTableReferences(final RexBuilder rexBuilder,
+      final RexNode node, final Map<RelTableRef, RelTableRef> tableMapping) {
+    return swapTableColumnReferences(rexBuilder, node, tableMapping, null);
+  }
+
+  public static RexNode swapColumnReferences(final RexBuilder rexBuilder,
+      final RexNode node, final Map<RexTableInputRef, Set<RexTableInputRef>> ec) {
+    return swapTableColumnReferences(rexBuilder, node, null, ec);
+  }
+
+  public static RexNode swapTableColumnReferences(final RexBuilder rexBuilder,
+      final RexNode node, final Map<RelTableRef, RelTableRef> tableMapping,
+      final Map<RexTableInputRef, Set<RexTableInputRef>> ec) {
+    RexShuttle visitor =
+        new RexShuttle() {
+          @Override public RexNode visitTableInputRef(RexTableInputRef inputRef) {
+            if (tableMapping != null) {
+              inputRef = new RexTableInputRef(
+                  tableMapping.get(inputRef.getTableRef()),
+                  inputRef.getIndex(),
+                  inputRef.getType());
+            }
+            if (ec != null) {
+              Set<RexTableInputRef> s = ec.get(inputRef);
+              if (s != null) {
+                inputRef = s.iterator().next();
+              }
+            }
+            return inputRef;
+          }
+        };
+    return visitor.apply(node);
+  }
+
+  /**
+   * Gather all table references in input expressions.
+   *
+   * @param nodes expressions
+   * @return set of table references
+   */
+  public static Set<RelTableRef> gatherTableReferences(final List<RexNode> nodes) {
+    final Set<RelTableRef> occurrences = new HashSet<>();
+    RexVisitor<Void> visitor =
+      new RexVisitorImpl<Void>(true) {
+        @Override public Void visitTableInputRef(RexTableInputRef ref) {
+          occurrences.add(ref.getTableRef());
+          return super.visitTableInputRef(ref);
+        }
+      };
+    for (RexNode e : nodes) {
+      e.accept(visitor);
+    }
+    return occurrences;
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 bf6dd99..a2a01d0 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexVisitor.java
@@ -48,6 +48,8 @@ public interface RexVisitor<R> {
 
   R visitSubQuery(RexSubQuery subQuery);
 
+  R visitTableInputRef(RexTableInputRef fieldRef);
+
   R visitPatternFieldRef(RexPatternFieldRef fieldRef);
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 4710f98..6e40a67 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexVisitorImpl.java
@@ -110,6 +110,10 @@ public class RexVisitorImpl<R> implements RexVisitor<R> {
     return r;
   }
 
+  @Override public R visitTableInputRef(RexTableInputRef ref) {
+    return null;
+  }
+
   @Override public R visitPatternFieldRef(RexPatternFieldRef fieldRef) {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 7a33772..07164b3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -561,6 +561,13 @@ public enum SqlKind {
   INPUT_REF,
 
   /**
+   * Reference to an input field, with a qualified name and an identifier
+   *
+   * <p>(Only used at the RexNode level.)</p>
+   */
+  TABLE_INPUT_REF,
+
+  /**
    * Reference to an input field, with pattern var as modifier
    *
    * <p>(Only used at the RexNode level.)</p>

http://git-wip-us.apache.org/repos/asf/calcite/blob/41b05d78/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 f5161b2..df2f2f4 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -43,6 +43,7 @@ import org.apache.calcite.linq4j.function.Predicate2;
 import org.apache.calcite.linq4j.tree.FunctionExpression;
 import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.metadata.BuiltInMetadata.AllPredicates;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.Collation;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.ColumnOrigin;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.ColumnUniqueness;
@@ -50,9 +51,11 @@ import org.apache.calcite.rel.metadata.BuiltInMetadata.CumulativeCost;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.DistinctRowCount;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.Distribution;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.ExplainVisibility;
+import org.apache.calcite.rel.metadata.BuiltInMetadata.ExpressionLineage;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.MaxRowCount;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.Memory;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.MinRowCount;
+import org.apache.calcite.rel.metadata.BuiltInMetadata.NodeTypes;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.NonCumulativeCost;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.Parallelism;
 import org.apache.calcite.rel.metadata.BuiltInMetadata.PercentageOriginalRows;
@@ -361,6 +364,7 @@ public enum BuiltInMethod {
       ImmutableBitSet.class, boolean.class),
   COLLATIONS(Collation.class, "collations"),
   DISTRIBUTION(Distribution.class, "distribution"),
+  NODE_TYPES(NodeTypes.class, "getNodeTypes"),
   ROW_COUNT(RowCount.class, "getRowCount"),
   MAX_ROW_COUNT(MaxRowCount.class, "getMaxRowCount"),
   MIN_ROW_COUNT(MinRowCount.class, "getMinRowCount"),
@@ -371,9 +375,11 @@ public enum BuiltInMethod {
   POPULATION_SIZE(PopulationSize.class, "getPopulationSize",
       ImmutableBitSet.class),
   COLUMN_ORIGIN(ColumnOrigin.class, "getColumnOrigins", int.class),
+  EXPRESSION_LINEAGE(ExpressionLineage.class, "getExpressionLineage", RexNode.class),
   CUMULATIVE_COST(CumulativeCost.class, "getCumulativeCost"),
   NON_CUMULATIVE_COST(NonCumulativeCost.class, "getNonCumulativeCost"),
   PREDICATES(Predicates.class, "getPredicates"),
+  ALL_PREDICATES(AllPredicates.class, "getAllPredicates"),
   EXPLAIN_VISIBILITY(ExplainVisibility.class, "isVisibleInExplain",
       SqlExplainLevel.class),
   SCALAR_EXECUTE1(Scalar.class, "execute", Context.class),


[03/11] calcite git commit: [CALCITE-1731] Materialized view rewriting for join and aggregate operators

Posted by jc...@apache.org.
[CALCITE-1731] Materialized view rewriting for join and aggregate operators


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

Branch: refs/heads/master
Commit: 84b49f5b48842708e9f4d912b4f9564745d74bc6
Parents: 41b05d7
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Mar 29 07:36:43 2017 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Apr 26 19:56:37 2017 +0100

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptCluster.java  |    1 -
 .../org/apache/calcite/plan/RelOptUtil.java     |    3 +-
 .../calcite/plan/SubstitutionVisitor.java       |   59 +-
 .../apache/calcite/plan/volcano/RelSubset.java  |    4 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |    5 +
 .../calcite/rel/metadata/BuiltInMetadata.java   |   78 +-
 .../metadata/DefaultRelMetadataProvider.java    |    1 +
 .../rel/metadata/RelMdExpressionLineage.java    |   18 +-
 .../rel/metadata/RelMdTableReferences.java      |  199 +++
 .../apache/calcite/rel/metadata/RelMdUtil.java  |    1 -
 .../calcite/rel/metadata/RelMetadataQuery.java  |   18 +
 .../calcite/rel/metadata/RelTableRef.java       |   62 -
 .../apache/calcite/rel/mutable/MutableRels.java |    9 +
 .../rel/rules/AbstractMaterializedViewRule.java | 1283 ++++++++++++++++++
 .../apache/calcite/rex/RexTableInputRef.java    |   70 +-
 .../java/org/apache/calcite/rex/RexUtil.java    |   19 +-
 .../org/apache/calcite/util/BuiltInMethod.java  |    2 +
 .../calcite/test/MaterializationTest.java       |  345 ++++-
 .../apache/calcite/test/RelMetadataTest.java    |   48 +-
 site/_docs/materialized_views.md                |   37 +-
 20 files changed, 2120 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 034d978..f88e232 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
@@ -37,7 +37,6 @@ import java.util.concurrent.atomic.AtomicInteger;
  * optimization of a query.
  */
 public class RelOptCluster {
-
   //~ Instance fields --------------------------------------------------------
 
   private final RelDataTypeFactory typeFactory;

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 662d316..b449417 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -240,7 +240,8 @@ public abstract class RelOptUtil {
   }
 
   /**
-   * Returns a list of all tables used by this expression or its children
+   * Returns a list of all table qualified names used by this expression
+   * or its children.
    */
   public static List<String> findAllTableQualifiedNames(RelNode rel) {
     return Lists.transform(findAllTables(rel),

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 bd6dddf..f7bf106 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -72,6 +72,7 @@ import org.slf4j.Logger;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -229,7 +230,7 @@ public class SubstitutionVisitor {
    * <ul>
    * <li>condition: x = 1</li>
    * <li>target: x = 1 OR z = 3</li>
-   * <li>residue: NOT (z = 3)</li>
+   * <li>residue: x = 1</li>
    * </ul>
    *
    * <p>Example #3: condition and target are equivalent</p>
@@ -258,32 +259,63 @@ public class SubstitutionVisitor {
     // First, try splitting into ORs.
     // Given target    c1 OR c2 OR c3 OR c4
     // and condition   c2 OR c4
-    // residue is      NOT c1 AND NOT c3
+    // residue is      c2 OR c4
     // Also deals with case target [x] condition [x] yields residue [true].
     RexNode z = splitOr(simplify.rexBuilder, condition, target);
     if (z != null) {
       return z;
     }
 
+    if (isEquivalent(simplify.rexBuilder, condition, target)) {
+      return simplify.rexBuilder.makeLiteral(true);
+    }
+
     RexNode x = andNot(simplify.rexBuilder, target, condition);
     if (mayBeSatisfiable(x)) {
-      RexNode x2 = andNot(simplify.rexBuilder, condition, target);
-      return simplify.simplify(x2);
+      RexNode x2 = RexUtil.composeConjunction(simplify.rexBuilder,
+          ImmutableList.of(condition, target), false);
+      RexNode r = simplify.withUnknownAsFalse(true).simplify(x2);
+      if (!r.isAlwaysFalse() && isEquivalent(simplify.rexBuilder, condition, r)) {
+        List<RexNode> conjs = RelOptUtil.conjunctions(r);
+        for (RexNode e : RelOptUtil.conjunctions(target)) {
+          removeAll(conjs, e);
+        }
+        return RexUtil.composeConjunction(simplify.rexBuilder, conjs, false);
+      }
     }
     return null;
   }
 
   private static RexNode splitOr(
       final RexBuilder rexBuilder, RexNode condition, RexNode target) {
-    List<RexNode> targets = RelOptUtil.disjunctions(target);
-    for (RexNode e : RelOptUtil.disjunctions(condition)) {
-      boolean found = removeAll(targets, e);
-      if (!found) {
-        return null;
-      }
+    List<RexNode> conditions = RelOptUtil.disjunctions(condition);
+    int conditionsLength = conditions.size();
+    int targetsLength = 0;
+    for (RexNode e : RelOptUtil.disjunctions(target)) {
+      removeAll(conditions, e);
+      targetsLength++;
+    }
+    if (conditions.isEmpty() && conditionsLength == targetsLength) {
+      return rexBuilder.makeLiteral(true);
+    } else if (conditions.isEmpty()) {
+      return condition;
     }
-    return RexUtil.composeConjunction(rexBuilder,
-        Lists.transform(targets, RexUtil.notFn(rexBuilder)), false);
+    return null;
+  }
+
+  private static boolean isEquivalent(RexBuilder rexBuilder, RexNode condition, RexNode target) {
+    // Example:
+    //  e: x = 1 AND y = 2 AND z = 3 AND NOT (x = 1 AND y = 2)
+    //  disjunctions: {x = 1, y = 2, z = 3}
+    //  notDisjunctions: {x = 1 AND y = 2}
+    final Set<String> conditionDisjunctions = new HashSet<>(
+        RexUtil.strings(RelOptUtil.conjunctions(condition)));
+    final Set<String> targetDisjunctions = new HashSet<>(
+        RexUtil.strings(RelOptUtil.conjunctions(target)));
+    if (conditionDisjunctions.equals(targetDisjunctions)) {
+      return true;
+    }
+    return false;
   }
 
   /**
@@ -1137,6 +1169,9 @@ public class SubstitutionVisitor {
       //   target: SELECT x, y, SUM(a) AS s, COUNT(b) AS cb FROM t GROUP BY x, y
       // transforms to
       //   result: SELECT x, SUM(cb) FROM (target) GROUP BY x
+      if (query.getInput() != target.getInput()) {
+        return null;
+      }
       if (!target.groupSet.contains(query.groupSet)) {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 af32671..24e15e8 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
@@ -35,8 +35,6 @@ import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
-import com.google.common.collect.Iterables;
-
 import org.slf4j.Logger;
 
 import java.io.PrintWriter;
@@ -168,7 +166,7 @@ public class RelSubset extends AbstractRelNode {
     String s = getDescription();
     pw.item("subset", s);
     final AbstractRelNode input =
-        (AbstractRelNode) Iterables.getFirst(getRels(), null);
+        (AbstractRelNode) Util.first(getBest(), getOriginal());
     if (input == null) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 9858b48..e0ae012 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -73,6 +73,7 @@ import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.rules.AbstractMaterializedViewRule;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
 import org.apache.calcite.rel.rules.AggregateStarTableRule;
@@ -533,6 +534,10 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     }
     if (prepareContext.config().materializationsEnabled()) {
       planner.addRule(MaterializedViewFilterScanRule.INSTANCE);
+      planner.addRule(AbstractMaterializedViewRule.INSTANCE_PROJECT_JOIN);
+      planner.addRule(AbstractMaterializedViewRule.INSTANCE_JOIN);
+      planner.addRule(AbstractMaterializedViewRule.INSTANCE_PROJECT_AGGREGATE);
+      planner.addRule(AbstractMaterializedViewRule.INSTANCE_AGGREGATE);
     }
     if (enableBindable) {
       for (RelOptRule rule : Bindables.RULES) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 0e0cbca..863e3e8 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
@@ -22,6 +22,8 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexTableInputRef.RelTableRef;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -161,13 +163,21 @@ public abstract class BuiltInMetadata {
     }
   }
 
-  /** Metadata about the node types and count in a relational expression. */
+  /**
+   * Metadata about the node types in a relational expression.
+   *
+   * <p>For each relational expression, it returns a multimap from the class
+   * to the nodes instantiating that class. Each node will appear in the
+   * multimap only once.
+   */
   public interface NodeTypes extends Metadata {
     MetadataDef<NodeTypes> DEF = MetadataDef.of(NodeTypes.class,
         NodeTypes.Handler.class, BuiltInMethod.NODE_TYPES.method);
 
     /**
-     *
+     * Returns a multimap from the class to the nodes instantiating that
+     * class. The default implementation for a node classifies it as a
+     * {@link RelNode}.
      */
     Multimap<Class<? extends RelNode>, RelNode> getNodeTypes();
 
@@ -393,7 +403,26 @@ public abstract class BuiltInMetadata {
         ExpressionLineage.Handler.class, BuiltInMethod.EXPRESSION_LINEAGE.method);
 
     /**
-     *
+     * Given the input expression applied on the given {@link RelNode}, this
+     * provider returns the expression with its lineage resolved.
+     *
+     * <p>In particular, the result will be a set of nodes which might contain
+     * references to columns in TableScan operators ({@link RexTableInputRef}).
+     * An expression can have more than one lineage expression due to Union
+     * operators. However, we do not check column equality in Filter predicates.
+     * Each TableScan operator below the node is identified uniquely by its
+     * qualified name and its entity number.
+     *
+     * <p>For example, if the expression is {@code $0 + 2} and {@code $0} originated
+     * from column {@code $3} in the {@code 0} occurrence of table {@code A} in the
+     * plan, result will be: {@code A.#0.$3 + 2}. Occurrences are generated in no
+     * particular order, but it is guaranteed that if two expressions referred to the
+     * same table, the qualified name + occurrence will be the same.
+     *
+     * @param expression expression whose lineage we want to resolve
+     * @return set of expressions with lineage resolved, or null if this information
+     * cannot be determined (e.g. origin of an expression is an aggregation
+     * in an {@link Aggregate} operator)
      */
     Set<RexNode> getExpressionLineage(RexNode expression);
 
@@ -404,6 +433,35 @@ public abstract class BuiltInMetadata {
     }
   }
 
+  /** Metadata to obtain references to tables used by a given expression. */
+  public interface TableReferences extends Metadata {
+    MetadataDef<TableReferences> DEF = MetadataDef.of(TableReferences.class,
+        TableReferences.Handler.class, BuiltInMethod.TABLE_REFERENCES.method);
+
+    /**
+     * This provider returns the tables used by a given plan.
+     *
+     * <p>In particular, the result will be a set of unique table references
+     * ({@link RelTableRef}) corresponding to each TableScan operator in the
+     * plan. These table references are composed by the table qualified name
+     * and an entity number.
+     *
+     * <p>Importantly, the table identifiers returned by this metadata provider
+     * will be consistent with the unique identifiers used by the {@link ExpressionLineage}
+     * provider, meaning that it is guaranteed that same table will use same unique
+     * identifiers in both.
+     *
+     * @return set of unique table identifiers, or null if this information
+     * cannot be determined
+     */
+    Set<RelTableRef> getTableReferences();
+
+    /** Handler API. */
+    interface Handler extends MetadataHandler<TableReferences> {
+      Set<RelTableRef> getTableReferences(RelNode r, RelMetadataQuery mq);
+    }
+  }
+
   /** Metadata about the cost of evaluating a relational expression, including
    * all of its inputs. */
   public interface CumulativeCost extends Metadata {
@@ -496,7 +554,14 @@ public abstract class BuiltInMetadata {
   }
 
   /** Metadata about the predicates that hold in the rows emitted from a
-   * relational expression. */
+   * relational expression.
+   *
+   * <p>The difference with respect to {@link Predicates} provider is that
+   * this provider tries to extract ALL predicates even if they are not
+   * applied on the output expressions of the relational expression; we rely
+   * on {@link RexTableInputRef} to reference origin columns in {@link TableScan}
+   * for the result predicates.
+   */
   public interface AllPredicates extends Metadata {
     MetadataDef<AllPredicates> DEF = MetadataDef.of(AllPredicates.class,
             AllPredicates.Handler.class, BuiltInMethod.ALL_PREDICATES.method);
@@ -505,7 +570,8 @@ public abstract class BuiltInMetadata {
      * Derives the predicates that hold on rows emitted from a relational
      * expression.
      *
-     * @return Predicate list
+     * @return predicate list, or null if the provider cannot infer the
+     * lineage for any of the expressions contained in any of the predicates
      */
     RelOptPredicateList getAllPredicates();
 
@@ -602,7 +668,7 @@ public abstract class BuiltInMetadata {
   interface All extends Selectivity, UniqueKeys, RowCount, DistinctRowCount,
       PercentageOriginalRows, ColumnUniqueness, ColumnOrigin, Predicates,
       Collation, Distribution, Size, Parallelism, Memory, AllPredicates,
-      ExpressionLineage, NodeTypes {
+      ExpressionLineage, TableReferences, NodeTypes {
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
index cb86698..74ef413 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/DefaultRelMetadataProvider.java
@@ -44,6 +44,7 @@ public class DefaultRelMetadataProvider extends ChainedRelMetadataProvider {
             RelMdPercentageOriginalRows.SOURCE,
             RelMdColumnOrigins.SOURCE,
             RelMdExpressionLineage.SOURCE,
+            RelMdTableReferences.SOURCE,
             RelMdNodeTypes.SOURCE,
             RelMdRowCount.SOURCE,
             RelMdMaxRowCount.SOURCE,

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
index 5f9e5ba..6ac5cfb 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdExpressionLineage.java
@@ -35,6 +35,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexTableInputRef.RelTableRef;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -65,7 +66,7 @@ import java.util.Set;
  *
  * The output expressions might contain references to columns produced by TableScan
  * operators ({@link RexTableInputRef}). In turn, each TableScan operator is identified
- * uniquely by its qualified name and an identifier contained in .
+ * uniquely by a {@link RelTableRef} containing its qualified name and an identifier.
  *
  * If the lineage cannot be inferred, we return null.
  */
@@ -122,7 +123,7 @@ public class RelMdExpressionLineage
     final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
     for (int idx : inputFieldsUsed) {
       final RexNode inputRef = RexTableInputRef.of(
-          new RelTableRef(rel.getTable().getQualifiedName().toString(), 0),
+          RelTableRef.of(rel.getTable().getQualifiedName(), 0),
           RexInputRef.of(idx, rel.getRowType().getFieldList()));
       final Set<RexNode> originalExprs = Sets.newHashSet(inputRef);
       final RexInputRef ref = RexInputRef.of(idx, rel.getRowType().getFieldList());
@@ -193,7 +194,7 @@ public class RelMdExpressionLineage
     final int nLeftColumns = leftInput.getRowType().getFieldList().size();
 
     // Infer column origin expressions for given references
-    final Multimap<String, RelTableRef> qualifiedNamesToRefs = HashMultimap.create();
+    final Multimap<List<String>, RelTableRef> qualifiedNamesToRefs = HashMultimap.create();
     final Map<RelTableRef, RelTableRef> currentTablesMapping = new HashMap<>();
     final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
     for (int idx = 0; idx < rel.getRowType().getFieldList().size(); idx++) {
@@ -232,7 +233,7 @@ public class RelMdExpressionLineage
             shift = lRefs.size();
           }
           currentTablesMapping.put(rightRef,
-              new RelTableRef(rightRef.getQualifiedName(), shift + rightRef.getIdentifier()));
+              RelTableRef.of(rightRef.getQualifiedName(), shift + rightRef.getEntityNumber()));
         }
         final Set<RexNode> updatedExprs = Sets.newHashSet(
             Iterables.transform(
@@ -262,7 +263,7 @@ public class RelMdExpressionLineage
     final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
 
     // Infer column origin expressions for given references
-    final Multimap<String, RelTableRef> qualifiedNamesToRefs = HashMultimap.create();
+    final Multimap<List<String>, RelTableRef> qualifiedNamesToRefs = HashMultimap.create();
     final Map<RexInputRef, Set<RexNode>> mapping = new LinkedHashMap<>();
     for (RelNode input : rel.getInputs()) {
       final Map<RelTableRef, RelTableRef> currentTablesMapping = new HashMap<>();
@@ -287,7 +288,7 @@ public class RelMdExpressionLineage
             shift = lRefs.size();
           }
           currentTablesMapping.put(tableRef,
-              new RelTableRef(tableRef.getQualifiedName(), shift + tableRef.getIdentifier()));
+              RelTableRef.of(tableRef.getQualifiedName(), shift + tableRef.getEntityNumber()));
         }
         final Set<RexNode> updatedExprs = Sets.newHashSet(
             Iterables.transform(
@@ -388,6 +389,11 @@ public class RelMdExpressionLineage
     expr.accept(inputFinder);
     final ImmutableBitSet predFieldsUsed = inputFinder.inputBitSet.build();
 
+    if (predFieldsUsed.isEmpty()) {
+      // The unique expression is the input expression
+      return Sets.newHashSet(expr);
+    }
+
     return createAllPossibleExpressions(rexBuilder, expr, predFieldsUsed, mapping,
         new HashMap<RexInputRef, RexNode>());
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java
new file mode 100644
index 0000000..358c872
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdTableReferences.java
@@ -0,0 +1,199 @@
+/*
+ * 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.plan.hep.HepRelVertex;
+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;
+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.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rex.RexTableInputRef.RelTableRef;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * RelMdTableReferences supplies a default implementation of
+ * {@link RelMetadataQuery#getTableReferences} for the standard logical algebra.
+ *
+ * The goal of this provider is to return all tables used by a given expression
+ * identified uniquely by a {@link RelTableRef}.
+ *
+ * Each unique identifier {@link RelTableRef} of a table will equal to the
+ * identifier obtained running {@link RelMdExpressionLineage} over the same
+ * plan node for an expression that refers to the same table.
+ *
+ * If tables cannot be obtained, we return null.
+ */
+public class RelMdTableReferences
+    implements MetadataHandler<BuiltInMetadata.TableReferences> {
+  public static final RelMetadataProvider SOURCE =
+      ReflectiveRelMetadataProvider.reflectiveSource(
+          BuiltInMethod.TABLE_REFERENCES.method, new RelMdTableReferences());
+
+  //~ Constructors -----------------------------------------------------------
+
+  private RelMdTableReferences() {}
+
+  //~ Methods ----------------------------------------------------------------
+
+  public MetadataDef<BuiltInMetadata.TableReferences> getDef() {
+    return BuiltInMetadata.TableReferences.DEF;
+  }
+
+  // Catch-all rule when none of the others apply.
+  public Set<RelTableRef> getTableReferences(RelNode rel, RelMetadataQuery mq) {
+    return null;
+  }
+
+  public Set<RelTableRef> getTableReferences(HepRelVertex rel, RelMetadataQuery mq) {
+    return mq.getTableReferences(rel.getCurrentRel());
+  }
+
+  public Set<RelTableRef> getTableReferences(RelSubset rel, RelMetadataQuery mq) {
+    return mq.getTableReferences(Util.first(rel.getBest(), rel.getOriginal()));
+  }
+
+  /**
+   * TableScan table reference.
+   */
+  public Set<RelTableRef> getTableReferences(TableScan rel, RelMetadataQuery mq) {
+    return Sets.newHashSet(RelTableRef.of(rel.getTable().getQualifiedName(), 0));
+  }
+
+  /**
+   * Table references from Aggregate.
+   */
+  public Set<RelTableRef> getTableReferences(Aggregate rel, RelMetadataQuery mq) {
+    return mq.getTableReferences(rel.getInput());
+  }
+
+  /**
+   * Table references from Join.
+   */
+  public Set<RelTableRef> getTableReferences(Join rel, RelMetadataQuery mq) {
+    final RelNode leftInput = rel.getLeft();
+    final RelNode rightInput = rel.getRight();
+    final Set<RelTableRef> result = new HashSet<>();
+
+    // Gather table references, left input references remain unchanged
+    final Multimap<List<String>, RelTableRef> leftQualifiedNamesToRefs = HashMultimap.create();
+    for (RelTableRef leftRef : mq.getTableReferences(leftInput)) {
+      assert !result.contains(leftRef);
+      result.add(leftRef);
+      leftQualifiedNamesToRefs.put(leftRef.getQualifiedName(), leftRef);
+    }
+
+    // Gather table references, right input references might need to be
+    // updated if there are table names clashes with left input
+    for (RelTableRef rightRef : mq.getTableReferences(rightInput)) {
+      int shift = 0;
+      Collection<RelTableRef> lRefs = leftQualifiedNamesToRefs.get(rightRef.getQualifiedName());
+      if (lRefs != null) {
+        shift = lRefs.size();
+      }
+      RelTableRef shiftTableRef = RelTableRef.of(
+          rightRef.getQualifiedName(), shift + rightRef.getEntityNumber());
+      assert !result.contains(shiftTableRef);
+      result.add(shiftTableRef);
+    }
+
+    // Return result
+    return result;
+  }
+
+  /**
+   * Table references from Union.
+   *
+   * For Union operator, we might be able to extract multiple table references.
+   */
+  public Set<RelTableRef> getTableReferences(Union rel, RelMetadataQuery mq) {
+    final Set<RelTableRef> result = new HashSet<>();
+
+    // Infer column origin expressions for given references
+    final Multimap<List<String>, RelTableRef> qualifiedNamesToRefs = HashMultimap.create();
+    for (RelNode input : rel.getInputs()) {
+      final Map<RelTableRef, RelTableRef> currentTablesMapping = new HashMap<>();
+      for (RelTableRef tableRef : mq.getTableReferences(input)) {
+        int shift = 0;
+        Collection<RelTableRef> lRefs = qualifiedNamesToRefs.get(
+            tableRef.getQualifiedName());
+        if (lRefs != null) {
+          shift = lRefs.size();
+        }
+        RelTableRef shiftTableRef = RelTableRef.of(
+            tableRef.getQualifiedName(), shift + tableRef.getEntityNumber());
+        assert !result.contains(shiftTableRef);
+        result.add(shiftTableRef);
+        currentTablesMapping.put(tableRef, shiftTableRef);
+      }
+      // Add to existing qualified names
+      for (RelTableRef newRef : currentTablesMapping.values()) {
+        qualifiedNamesToRefs.put(newRef.getQualifiedName(), newRef);
+      }
+    }
+
+    // Return result
+    return result;
+  }
+
+  /**
+   * Table references from Project.
+   */
+  public Set<RelTableRef> getTableReferences(Project rel, final RelMetadataQuery mq) {
+    return mq.getTableReferences(rel.getInput());
+  }
+
+  /**
+   * Table references from Filter.
+   */
+  public Set<RelTableRef> getTableReferences(Filter rel, RelMetadataQuery mq) {
+    return mq.getTableReferences(rel.getInput());
+  }
+
+  /**
+   * Table references from Sort.
+   */
+  public Set<RelTableRef> getTableReferences(Sort rel, RelMetadataQuery mq) {
+    return mq.getTableReferences(rel.getInput());
+  }
+
+  /**
+   * Table references from Exchange.
+   */
+  public Set<RelTableRef> getTableReferences(Exchange rel, RelMetadataQuery mq) {
+    return mq.getTableReferences(rel.getInput());
+  }
+}
+
+// End RelMdTableReferences.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 4ef5592..7b63ac9 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
@@ -843,7 +843,6 @@ public class RelMdUtil {
     }
     return alreadySorted && alreadySmaller;
   }
-
 }
 
 // End RelMdUtil.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/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 9dd19dc..7298ba3 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
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexTableInputRef.RelTableRef;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.ImmutableBitSet;
 
@@ -86,6 +87,7 @@ public class RelMetadataQuery {
   private BuiltInMetadata.Collation.Handler collationHandler;
   private BuiltInMetadata.ColumnOrigin.Handler columnOriginHandler;
   private BuiltInMetadata.ExpressionLineage.Handler expressionLineageHandler;
+  private BuiltInMetadata.TableReferences.Handler tableReferencesHandler;
   private BuiltInMetadata.ColumnUniqueness.Handler columnUniquenessHandler;
   private BuiltInMetadata.CumulativeCost.Handler cumulativeCostHandler;
   private BuiltInMetadata.DistinctRowCount.Handler distinctRowCountHandler;
@@ -119,6 +121,7 @@ public class RelMetadataQuery {
     this.collationHandler = prototype.collationHandler;
     this.columnOriginHandler = prototype.columnOriginHandler;
     this.expressionLineageHandler = prototype.expressionLineageHandler;
+    this.tableReferencesHandler = prototype.tableReferencesHandler;
     this.columnUniquenessHandler = prototype.columnUniquenessHandler;
     this.cumulativeCostHandler = prototype.cumulativeCostHandler;
     this.distinctRowCountHandler = prototype.distinctRowCountHandler;
@@ -170,6 +173,7 @@ public class RelMetadataQuery {
     this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
     this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
     this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
+    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
     this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
     this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
     this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
@@ -394,6 +398,20 @@ public class RelMetadataQuery {
   }
 
   /**
+   * Determines the tables used by a plan.
+   */
+  public Set<RelTableRef> getTableReferences(RelNode rel) {
+    for (;;) {
+      try {
+        return tableReferencesHandler.getTableReferences(rel, this);
+      } catch (JaninoRelMetadataProvider.NoHandler e) {
+        tableReferencesHandler =
+            revise(e.relClass, BuiltInMetadata.TableReferences.DEF);
+      }
+    }
+  }
+
+  /**
    * Determines the origin of a {@link RelNode}, provided it maps to a single
    * table, optionally with filtering and projection.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/core/src/main/java/org/apache/calcite/rel/metadata/RelTableRef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelTableRef.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelTableRef.java
deleted file mode 100644
index bdd032f..0000000
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelTableRef.java
+++ /dev/null
@@ -1,62 +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.metadata;
-
-/**
- *
- *
- */
-public class RelTableRef {
-
-  private final String qualifiedName;
-  private final int identifier;
-  private final String digest;
-
-  public RelTableRef(String qualifiedName, int identifier) {
-    this.qualifiedName = qualifiedName;
-    this.identifier = identifier;
-    this.digest = qualifiedName + ".#" + identifier;
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  @Override public boolean equals(Object obj) {
-    return this == obj
-        || obj instanceof RelTableRef
-        && qualifiedName.equals(((RelTableRef) obj).qualifiedName)
-        && identifier == ((RelTableRef) obj).identifier;
-  }
-
-  @Override public int hashCode() {
-    return digest.hashCode();
-  }
-
-  public String getQualifiedName() {
-    return qualifiedName;
-  }
-
-  public int getIdentifier() {
-    return identifier;
-  }
-
-  @Override public String toString() {
-    return digest;
-  }
-
-}
-
-// End RelTableRef.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/84b49f5b/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java b/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
index 04b1849..ed1f1b1 100644
--- a/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
+++ b/core/src/main/java/org/apache/calcite/rel/mutable/MutableRels.java
@@ -17,6 +17,8 @@
 package org.apache.calcite.rel.mutable;
 
 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;
 import org.apache.calcite.rel.core.Calc;
@@ -271,6 +273,13 @@ public abstract class MutableRels {
   }
 
   public static MutableRel toMutable(RelNode rel) {
+    if (rel instanceof HepRelVertex) {
+      return toMutable(((HepRelVertex) rel).getCurrentRel());
+    }
+    if (rel instanceof RelSubset) {
+      return toMutable(
+          Util.first(((RelSubset) rel).getBest(), ((RelSubset) rel).getOriginal()));
+    }
     if (rel instanceof TableScan) {
       return MutableScan.of((TableScan) rel);
     }


[08/11] calcite git commit: [CALCITE-1760] Implement utility method to identify lossless casts

Posted by jc...@apache.org.
[CALCITE-1760] Implement utility method to identify lossless casts


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

Branch: refs/heads/master
Commit: a2bd49c32b6b08f92cf92fd603f8448684044431
Parents: 27ca310
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Apr 25 17:11:04 2017 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Apr 26 20:03:11 2017 +0100

----------------------------------------------------------------------
 .../java/org/apache/calcite/rex/RexUtil.java    | 43 +++++++++
 .../org/apache/calcite/test/RexProgramTest.java | 99 ++++++++++++++++++++
 2 files changed, 142 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/a2bd49c3/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 95ab2c2..26c1268 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -35,6 +35,7 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
@@ -1365,6 +1366,48 @@ public class RexUtil {
     }
   }
 
+  /**
+   * Returns whether the input is a 'lossless' casts, i.e., a cast from which the original
+   * value of the field can be certainly recovered.
+   *
+   * <p>For instance, int -> bigint is true (as you can cast back to int without loss of
+   * information), however bigint -> int is false.
+   *
+   * <p>The implementation of this method does not return false positives. However, it is
+   * not complete.
+   */
+  public static boolean isLosslessCast(RexNode node) {
+    if (!node.isA(SqlKind.CAST)) {
+      return false;
+    }
+    final RelDataType source = ((RexCall) node).getOperands().get(0).getType();
+    final SqlTypeName sourceSqlTypeName = source.getSqlTypeName();
+    final RelDataType target = node.getType();
+    final SqlTypeName targetSqlTypeName = target.getSqlTypeName();
+    // 1) Both INT numeric types
+    if (SqlTypeFamily.INTEGER.getTypeNames().contains(sourceSqlTypeName)
+        && SqlTypeFamily.INTEGER.getTypeNames().contains(targetSqlTypeName)) {
+      return targetSqlTypeName.compareTo(sourceSqlTypeName) >= 0;
+    }
+    // 2) Both CHARACTER types: it depends on the precision (length)
+    if (SqlTypeFamily.CHARACTER.getTypeNames().contains(sourceSqlTypeName)
+        && SqlTypeFamily.CHARACTER.getTypeNames().contains(targetSqlTypeName)) {
+      return targetSqlTypeName.compareTo(sourceSqlTypeName) >= 0
+          && source.getPrecision() <= target.getPrecision();
+    }
+    // 3) From NUMERIC family to CHARACTER family: it depends on the precision/scale
+    if (sourceSqlTypeName.getFamily() == SqlTypeFamily.NUMERIC
+        && targetSqlTypeName.getFamily() == SqlTypeFamily.CHARACTER) {
+      int sourceLength = source.getPrecision() + 1; // include sign
+      if (source.getScale() != -1 && source.getScale() != 0) {
+        sourceLength += source.getScale() + 1; // include decimal mark
+      }
+      return target.getPrecision() >= sourceLength;
+    }
+    // Return FALSE by default
+    return false;
+  }
+
   /** Converts an expression to conjunctive normal form (CNF).
    *
    * <p>The following expression is in CNF:

http://git-wip-us.apache.org/repos/asf/calcite/blob/a2bd49c3/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 ebc17f6..f2f6c68 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -634,6 +634,105 @@ public class RexProgramTest {
 
   }
 
+  /** Unit test for {@link org.apache.calcite.rex.RexUtil#isLosslessCast(RexNode)}. */
+  @Test public void testLosslessCast() {
+    final RelDataType tinyIntType = typeFactory.createSqlType(SqlTypeName.TINYINT);
+    final RelDataType smallIntType = typeFactory.createSqlType(SqlTypeName.SMALLINT);
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    final RelDataType bigIntType = typeFactory.createSqlType(SqlTypeName.BIGINT);
+    final RelDataType floatType = typeFactory.createSqlType(SqlTypeName.FLOAT);
+    final RelDataType booleanType = typeFactory.createSqlType(SqlTypeName.BOOLEAN);
+    final RelDataType charType5 = typeFactory.createSqlType(SqlTypeName.CHAR, 5);
+    final RelDataType charType6 = typeFactory.createSqlType(SqlTypeName.CHAR, 6);
+    final RelDataType varCharType10 = typeFactory.createSqlType(SqlTypeName.VARCHAR, 10);
+    final RelDataType varCharType11 = typeFactory.createSqlType(SqlTypeName.VARCHAR, 11);
+
+    // Negative
+    assertThat(RexUtil.isLosslessCast(rexBuilder.makeInputRef(intType, 0)), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                tinyIntType, rexBuilder.makeInputRef(smallIntType, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                smallIntType, rexBuilder.makeInputRef(intType, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                intType, rexBuilder.makeInputRef(bigIntType, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                bigIntType, rexBuilder.makeInputRef(floatType, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                booleanType, rexBuilder.makeInputRef(bigIntType, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                intType, rexBuilder.makeInputRef(charType5, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                intType, rexBuilder.makeInputRef(varCharType10, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                varCharType10, rexBuilder.makeInputRef(varCharType11, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                charType5, rexBuilder.makeInputRef(bigIntType, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                charType5, rexBuilder.makeInputRef(smallIntType, 0))), is(false));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                varCharType10, rexBuilder.makeInputRef(intType, 0))), is(false));
+
+    // Positive
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                smallIntType, rexBuilder.makeInputRef(tinyIntType, 0))), is(true));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                intType, rexBuilder.makeInputRef(smallIntType, 0))), is(true));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                bigIntType, rexBuilder.makeInputRef(intType, 0))), is(true));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                intType, rexBuilder.makeInputRef(intType, 0))), is(true));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                charType6, rexBuilder.makeInputRef(smallIntType, 0))), is(true));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                varCharType10, rexBuilder.makeInputRef(smallIntType, 0))), is(true));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                varCharType11, rexBuilder.makeInputRef(intType, 0))), is(true));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                varCharType11, rexBuilder.makeInputRef(charType6, 0))), is(true));
+    assertThat(
+        RexUtil.isLosslessCast(
+            rexBuilder.makeCast(
+                varCharType11, rexBuilder.makeInputRef(varCharType10, 0))), is(true));
+  }
+
   /** Unit test for {@link org.apache.calcite.rex.RexUtil#toCnf}. */
   @Test public void testCnf() {
     final RelDataType booleanType =