You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2017/03/09 20:13:19 UTC

phoenix git commit: PHOENIX-3724 Infinite loop in local index tests having filters on non indexed columns in Phoenix-Calcite

Repository: phoenix
Updated Branches:
  refs/heads/calcite c945b5017 -> 6f23af898


PHOENIX-3724 Infinite loop in local index tests having filters on non indexed columns in Phoenix-Calcite


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

Branch: refs/heads/calcite
Commit: 6f23af89818f3f0ba8b6de4b68f948e4d501afda
Parents: c945b50
Author: maryannxue <ma...@gmail.com>
Authored: Thu Mar 9 12:11:53 2017 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Thu Mar 9 12:11:53 2017 -0800

----------------------------------------------------------------------
 .../phoenix/calcite/rel/PhoenixTableScan.java      |  7 +++----
 .../calcite/rules/PhoenixFilterScanMergeRule.java  | 17 ++++++++++++++---
 .../rules/PhoenixTableScanColumnRefRule.java       | 13 +++++++++++--
 3 files changed, 28 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f23af89/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java
index f57f974..234ac5d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java
@@ -115,10 +115,9 @@ public class PhoenixTableScan extends TableScan implements PhoenixQueryRel {
             if (extendedColumnRef == null) {
                 extendedColumnRef = tableMapping.getDefaultExtendedColumnRef();
             }
-            if (filter == null) {
-                this.extendedColumnRef = extendedColumnRef;
-            } else {
-                this.extendedColumnRef = extendedColumnRef.union(
+            this.extendedColumnRef = extendedColumnRef;
+            if (filter != null) {
+                assert this.extendedColumnRef.contains(
                         tableMapping.getExtendedColumnRef(ImmutableList.of(filter)));
                 // We use a implementor with a special implementation for correlate variables
                 // or bind parameters here, which translates them into a LiteralExpression

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f23af89/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixFilterScanMergeRule.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixFilterScanMergeRule.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixFilterScanMergeRule.java
index e932fad..d13109b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixFilterScanMergeRule.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixFilterScanMergeRule.java
@@ -1,11 +1,16 @@
 package org.apache.phoenix.calcite.rules;
 
 import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
 
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.phoenix.calcite.PhoenixTable;
+import org.apache.phoenix.calcite.TableMapping;
 import org.apache.phoenix.calcite.rel.PhoenixTableScan;
 import org.apache.phoenix.calcite.rel.PhoenixTableScan.ScanOrder;
 
@@ -42,9 +47,15 @@ public class PhoenixFilterScanMergeRule extends RelOptRule {
     public void onMatch(RelOptRuleCall call) {
         LogicalFilter filter = call.rel(0);
         PhoenixTableScan scan = call.rel(1);
-        assert scan.filter == null : "predicate should have ensured no filter";
+        RexNode condition = filter.getCondition();
+        TableMapping tableMapping =
+                scan.getTable().unwrap(PhoenixTable.class).tableMapping;
+        ImmutableBitSet columnRef = 
+                scan.extendedColumnRef.union(
+                        tableMapping.getExtendedColumnRef(
+                                ImmutableList.of(condition)));
         call.transformTo(PhoenixTableScan.create(
-                scan.getCluster(), scan.getTable(), filter.getCondition(),
-                scan.scanOrder, scan.extendedColumnRef));
+                scan.getCluster(), scan.getTable(),
+                condition, scan.scanOrder, columnRef));
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f23af89/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixTableScanColumnRefRule.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixTableScanColumnRefRule.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixTableScanColumnRefRule.java
index 9076593..aa0fae8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixTableScanColumnRefRule.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixTableScanColumnRefRule.java
@@ -5,8 +5,10 @@ import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.phoenix.calcite.PhoenixTable;
+import org.apache.phoenix.calcite.TableMapping;
 import org.apache.phoenix.calcite.rel.PhoenixTableScan;
 import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
 
 public class PhoenixTableScanColumnRefRule extends RelOptRule {
 
@@ -32,8 +34,15 @@ public class PhoenixTableScanColumnRefRule extends RelOptRule {
     public void onMatch(RelOptRuleCall call) {
         Project project = call.rel(0);
         PhoenixTableScan scan = call.rel(1);
-        ImmutableBitSet bitSet = scan.getTable().unwrap(PhoenixTable.class)
-                .tableMapping.getExtendedColumnRef(project.getProjects());
+        TableMapping tableMapping =
+                scan.getTable().unwrap(PhoenixTable.class).tableMapping;
+        ImmutableBitSet bitSet =
+                tableMapping.getExtendedColumnRef(project.getProjects());
+        if (scan.filter != null) {
+            bitSet = bitSet.union(
+                    tableMapping.getExtendedColumnRef(
+                            ImmutableList.of(scan.filter)));
+        }
         if (bitSet.contains(scan.extendedColumnRef)) {
             return;
         }