You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/10/12 19:51:10 UTC

[02/10] hive git commit: HIVE-12021: HivePreFilteringRule may introduce wrong common operands (Jesus Camacho Rodriguez, reviewed by Laljo John Pullokkaran)

HIVE-12021: HivePreFilteringRule may introduce wrong common operands (Jesus Camacho Rodriguez, reviewed by Laljo John Pullokkaran)


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

Branch: refs/heads/llap
Commit: be05e32e58f54c0185551cb61d9769d2ceb5bbdd
Parents: 5201f18
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Sun Oct 4 11:40:30 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Oct 9 09:04:49 2015 +0100

----------------------------------------------------------------------
 .../calcite/rules/HivePreFilteringRule.java     | 58 ++++++++------
 .../clientpositive/filter_cond_pushdown.q       |  5 ++
 .../clientpositive/filter_cond_pushdown.q.out   | 80 ++++++++++++++++++++
 3 files changed, 121 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/be05e32e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
index 349c7f8..5824127 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
 
 
 public class HivePreFilteringRule extends RelOptRule {
@@ -178,54 +179,67 @@ public class HivePreFilteringRule extends RelOptRule {
     assert condition.getKind() == SqlKind.OR;
     Multimap<String,RexNode> reductionCondition = LinkedHashMultimap.create();
 
+    // Data structure to control whether a certain reference is present in every operand
+    Set<String> refsInAllOperands = null;
+
     // 1. We extract the information necessary to create the predicate for the new
     //    filter; currently we support comparison functions, in and between
     ImmutableList<RexNode> operands = RexUtil.flattenOr(((RexCall) condition).getOperands());
-    for (RexNode operand: operands) {
+    for (int i = 0; i < operands.size(); i++) {
+      final RexNode operand = operands.get(i);
+
       final RexNode operandCNF = RexUtil.toCnf(rexBuilder, operand);
       final List<RexNode> conjunctions = RelOptUtil.conjunctions(operandCNF);
-      boolean addedToReductionCondition = false; // Flag to control whether we have added a new factor
-                                                 // to the reduction predicate
+
+      Set<String> refsInCurrentOperand = Sets.newHashSet();
       for (RexNode conjunction: conjunctions) {
+        // We do not know what it is, we bail out for safety
         if (!(conjunction instanceof RexCall)) {
-          continue;
+          return new ArrayList<>();
         }
         RexCall conjCall = (RexCall) conjunction;
+        RexNode ref = null;
         if(COMPARISON.contains(conjCall.getOperator().getKind())) {
           if (conjCall.operands.get(0) instanceof RexInputRef &&
                   conjCall.operands.get(1) instanceof RexLiteral) {
-            reductionCondition.put(conjCall.operands.get(0).toString(),
-                    conjCall);
-            addedToReductionCondition = true;
+            ref = conjCall.operands.get(0);
           } else if (conjCall.operands.get(1) instanceof RexInputRef &&
                   conjCall.operands.get(0) instanceof RexLiteral) {
-            reductionCondition.put(conjCall.operands.get(1).toString(),
-                    conjCall);
-            addedToReductionCondition = true;
+            ref = conjCall.operands.get(1);
+          } else {
+            // We do not know what it is, we bail out for safety
+            return new ArrayList<>();
           }
         } else if(conjCall.getOperator().getKind().equals(SqlKind.IN)) {
-          reductionCondition.put(conjCall.operands.get(0).toString(),
-                  conjCall);
-          addedToReductionCondition = true;
+          ref = conjCall.operands.get(0);
         } else if(conjCall.getOperator().getKind().equals(SqlKind.BETWEEN)) {
-          reductionCondition.put(conjCall.operands.get(1).toString(),
-                  conjCall);
-          addedToReductionCondition = true;
+          ref = conjCall.operands.get(1);
+        } else {
+          // We do not know what it is, we bail out for safety
+          return new ArrayList<>();
         }
+
+        String stringRef = ref.toString();
+        reductionCondition.put(stringRef, conjCall);
+        refsInCurrentOperand.add(stringRef);
       }
 
-      // If we did not add any factor, we can bail out
-      if (!addedToReductionCondition) {
+      // Updates the references that are present in every operand up till now
+      if (i == 0) {
+        refsInAllOperands = refsInCurrentOperand;
+      } else {
+        refsInAllOperands = Sets.intersection(refsInAllOperands, refsInCurrentOperand);
+      }
+      // If we did not add any factor or there are no common factors, we can bail out
+      if (refsInAllOperands.isEmpty()) {
         return new ArrayList<>();
       }
     }
 
     // 2. We gather the common factors and return them
     List<RexNode> commonOperands = new ArrayList<>();
-    for (Entry<String,Collection<RexNode>> pair : reductionCondition.asMap().entrySet()) {
-      if (pair.getValue().size() == operands.size()) {
-        commonOperands.add(RexUtil.composeDisjunction(rexBuilder, pair.getValue(), false));
-      }
+    for (String ref : refsInAllOperands) {
+      commonOperands.add(RexUtil.composeDisjunction(rexBuilder, reductionCondition.get(ref), false));
     }
     return commonOperands;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/be05e32e/ql/src/test/queries/clientpositive/filter_cond_pushdown.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/filter_cond_pushdown.q b/ql/src/test/queries/clientpositive/filter_cond_pushdown.q
index 5e23b71..2425706 100644
--- a/ql/src/test/queries/clientpositive/filter_cond_pushdown.q
+++ b/ql/src/test/queries/clientpositive/filter_cond_pushdown.q
@@ -17,3 +17,8 @@ JOIN (
   JOIN (SELECT * FROM cbo_t3 t3 WHERE c_int=1) t3 ON t2.key=t3.c_int
   WHERE ((t2.key=t3.key) AND (t2.c_float + t3.c_float > 2)) OR
       ((t2.key=t3.key) AND (t2.c_int + t3.c_int > 2))) t4 ON t1.key=t4.key;
+
+EXPLAIN
+SELECT f.key, f.value, m.value
+FROM src f JOIN src m ON(f.key = m.key AND m.value is not null AND m.value !='')
+WHERE (f.value IN ('2008-04-08','2008-04-10') AND f.value IN ('2008-04-08','2008-04-09') AND m.value='2008-04-10') OR (m.value='2008-04-08');

http://git-wip-us.apache.org/repos/asf/hive/blob/be05e32e/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out b/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out
index af42d5c..99eb3f7 100644
--- a/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/filter_cond_pushdown.q.out
@@ -380,3 +380,83 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+PREHOOK: query: EXPLAIN
+SELECT f.key, f.value, m.value
+FROM src f JOIN src m ON(f.key = m.key AND m.value is not null AND m.value !='')
+WHERE (f.value IN ('2008-04-08','2008-04-10') AND f.value IN ('2008-04-08','2008-04-09') AND m.value='2008-04-10') OR (m.value='2008-04-08')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT f.key, f.value, m.value
+FROM src f JOIN src m ON(f.key = m.key AND m.value is not null AND m.value !='')
+WHERE (f.value IN ('2008-04-08','2008-04-10') AND f.value IN ('2008-04-08','2008-04-09') AND m.value='2008-04-10') OR (m.value='2008-04-08')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: f
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+          TableScan
+            alias: f
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((((value = '2008-04-10') or (value = '2008-04-08')) and value is not null) and (value <> '')) and key is not null) (type: boolean)
+              Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col3
+          Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: (((_col1) IN ('2008-04-08', '2008-04-10') and (_col1) IN ('2008-04-08', '2008-04-09') and (_col3 = '2008-04-10')) or (_col3 = '2008-04-08')) (type: boolean)
+            Statistics: Num rows: 171 Data size: 1816 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 171 Data size: 1816 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 171 Data size: 1816 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+