You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2016/12/06 17:47:20 UTC

hive git commit: HIVE-15369: Extend column pruner to account for residual filter expression in Join operator (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master d90d63859 -> f9141094e


HIVE-15369: Extend column pruner to account for residual filter expression in Join operator (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: f9141094e10d758ac9189907bf1318866c2fb8f2
Parents: d90d638
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Dec 6 16:00:26 2016 +0000
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Tue Dec 6 17:47:07 2016 +0000

----------------------------------------------------------------------
 .../ql/optimizer/ColumnPrunerProcFactory.java   |  16 ++-
 ql/src/test/queries/clientpositive/join46.q     |  12 ++
 ql/src/test/results/clientpositive/join46.q.out |  98 ++++++++++++++++
 .../results/clientpositive/llap/join46.q.out    | 112 +++++++++++++++++++
 4 files changed, 234 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f9141094/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
index 7681a83..00ec03e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
@@ -1083,11 +1083,12 @@ public final class ColumnPrunerProcFactory {
 
     LOG.info("JOIN " + op.getIdentifier() + " oldExprs: " + conf.getExprs());
 
-    List<FieldNode> childColLists = cppCtx.genColLists(op);
-    if (childColLists == null) {
+    if (cppCtx.genColLists(op) == null) {
       return;
     }
 
+    List<FieldNode> neededColList = new ArrayList<>(cppCtx.genColLists(op));
+
     Map<Byte, List<FieldNode>> prunedColLists = new HashMap<>();
     for (byte tag : conf.getTagOrder()) {
       prunedColLists.put(tag, new ArrayList<FieldNode>());
@@ -1107,6 +1108,13 @@ public final class ColumnPrunerProcFactory {
      }
     }
 
+    //add the columns in residual filters
+    if (conf.getResidualFilterExprs() != null) {
+      for (ExprNodeDesc desc : conf.getResidualFilterExprs()) {
+        neededColList = mergeFieldNodesWithDesc(neededColList, desc);
+      }
+    }
+
     RowSchema joinRS = op.getSchema();
     ArrayList<String> outputCols = new ArrayList<String>();
     ArrayList<ColumnInfo> rs = new ArrayList<ColumnInfo>();
@@ -1116,7 +1124,7 @@ public final class ColumnPrunerProcFactory {
       String internalName = conf.getOutputColumnNames().get(i);
       ExprNodeDesc desc = columnExprMap.get(internalName);
       Byte tag = conf.getReversedExprs().get(internalName);
-      if (lookupColumn(childColLists, internalName) == null) {
+      if (lookupColumn(neededColList, internalName) == null) {
         int index = conf.getExprs().get(tag).indexOf(desc);
         if (index < 0) {
           continue;
@@ -1174,7 +1182,7 @@ public final class ColumnPrunerProcFactory {
 
     for (Operator<? extends OperatorDesc> child : childOperators) {
       if (child instanceof ReduceSinkOperator) {
-        boolean[] flags = getPruneReduceSinkOpRetainFlags(toColumnNames(childColLists),
+        boolean[] flags = getPruneReduceSinkOpRetainFlags(toColumnNames(neededColList),
             (ReduceSinkOperator) child);
         pruneReduceSinkOperator(flags, (ReduceSinkOperator) child, cppCtx);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/f9141094/ql/src/test/queries/clientpositive/join46.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/join46.q b/ql/src/test/queries/clientpositive/join46.q
index e9e9850..a661c0f 100644
--- a/ql/src/test/queries/clientpositive/join46.q
+++ b/ql/src/test/queries/clientpositive/join46.q
@@ -118,6 +118,18 @@ ON (test1.value=test2.value
   AND (test1.key between 100 and 102
     OR test2.key between 100 and 102));
 
+-- Complex condition, projecting columns
+EXPLAIN
+SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key);
+
+SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key);
+
 -- Disjunction with pred on multiple inputs and single inputs (right outer join)
 EXPLAIN
 SELECT *

http://git-wip-us.apache.org/repos/asf/hive/blob/f9141094/ql/src/test/results/clientpositive/join46.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join46.q.out b/ql/src/test/results/clientpositive/join46.q.out
index 90f0620..21ae29c 100644
--- a/ql/src/test/results/clientpositive/join46.q.out
+++ b/ql/src/test/results/clientpositive/join46.q.out
@@ -912,6 +912,104 @@ NULL	NULL	None	NULL	NULL	NULL
 101	2	Car	102	2	Del
 99	2	Mat	102	2	Del
 Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Complex condition, projecting columns
+EXPLAIN
+SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Complex condition, projecting columns
+EXPLAIN
+SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key)
+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: test1
+            Statistics: Num rows: 6 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: int), value (type: int), col_1 (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 6 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 6 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string)
+          TableScan
+            alias: test2
+            Statistics: Num rows: 4 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: int), value (type: int), col_2 (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 4 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 4 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          residual filter predicates: {((_col1 = _col4) or (_col0 = _col3))}
+          Statistics: Num rows: 24 Data size: 476 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col2 (type: string), _col5 (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 24 Data size: 476 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 24 Data size: 476 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+PREHOOK: Input: default@test2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test1
+POSTHOOK: Input: default@test2
+#### A masked pattern was here ####
+Car	Ema
+Car	Del
+Bob	NULL
+Mat	Ema
+Mat	Del
+Alice	NULL
+None	NULL
+None	NULL
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- Disjunction with pred on multiple inputs and single inputs (right outer join)
 EXPLAIN
 SELECT *

http://git-wip-us.apache.org/repos/asf/hive/blob/f9141094/ql/src/test/results/clientpositive/llap/join46.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/join46.q.out b/ql/src/test/results/clientpositive/llap/join46.q.out
index fe1cf37..741f0e3 100644
--- a/ql/src/test/results/clientpositive/llap/join46.q.out
+++ b/ql/src/test/results/clientpositive/llap/join46.q.out
@@ -1038,6 +1038,118 @@ NULL	NULL	None	NULL	NULL	NULL
 101	2	Car	102	2	Del
 101	2	Car	103	2	Ema
 Warning: Shuffle Join MERGEJOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: -- Complex condition, projecting columns
+EXPLAIN
+SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Complex condition, projecting columns
+EXPLAIN
+SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: test1
+                  Statistics: Num rows: 6 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: key (type: int), value (type: int), col_1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 6 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 6 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: test2
+                  Statistics: Num rows: 4 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: key (type: int), value (type: int), col_2 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 4 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 4 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                residual filter predicates: {((_col1 = _col4) or (_col0 = _col3))}
+                Statistics: Num rows: 24 Data size: 476 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col2 (type: string), _col5 (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 24 Data size: 476 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 24 Data size: 476 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+PREHOOK: Input: default@test2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT col_1, col_2
+FROM test1 LEFT OUTER JOIN test2
+ON (test1.value=test2.value
+      OR test1.key=test2.key)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test1
+POSTHOOK: Input: default@test2
+#### A masked pattern was here ####
+None	NULL
+None	NULL
+Alice	NULL
+Mat	Del
+Mat	Ema
+Bob	NULL
+Car	Del
+Car	Ema
+Warning: Shuffle Join MERGEJOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: -- Disjunction with pred on multiple inputs and single inputs (right outer join)
 EXPLAIN
 SELECT *