You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2015/11/30 20:15:29 UTC

[18/27] hive git commit: HIVE-12503 : GBY-Join transpose rule may go in infinite loop (Ashutosh Chauhan via Jesus Camacho Rodriguez)

HIVE-12503 : GBY-Join transpose rule may go in infinite loop (Ashutosh Chauhan via Jesus Camacho Rodriguez)


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

Branch: refs/heads/master-fixed
Commit: a8e61c27b63e7cacbd817e848a06a17ee2208132
Parents: 18ca715
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Thu Nov 26 11:39:50 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Mon Nov 30 11:14:37 2015 -0800

----------------------------------------------------------------------
 .../rules/HiveAggregateJoinTransposeRule.java   |  17 ++-
 .../queries/clientpositive/cbo_rp_auto_join1.q  |   2 +-
 .../clientpositive/cbo_rp_auto_join1.q.out      | 125 +++++++++++++-----
 .../clientpositive/groupby_join_pushdown.q.out  | 128 +++++--------------
 4 files changed, 138 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a8e61c27/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
index c59af39..8cbaed0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
@@ -17,6 +17,7 @@
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
@@ -295,15 +296,13 @@ public class HiveAggregateJoinTransposeRule extends AggregateJoinTransposeRule {
           Mappings.apply(mapping, aggregate.getGroupSet()),
           Mappings.apply2(mapping, aggregate.getGroupSets()), newAggCalls);
     }
-    call.transformTo(r);
-    // Add original tree as well for potential alternative transformation.
-    // This is modeled after LoptOptimizeJoinRule::findBestOrderings() in
-    // which rule adds multiple transformations and Planner picks the cheapest one.
-    // Hep planner will automatically pick the one with lower cost among two.
-    // For details, see: HepPlanner:applyTransformationResults()
-    // In this case, if ndv is close to # of rows, i.e., group by is not resulting
-    // in any deduction, doing this transformation is not useful.
-    call.transformTo(aggregate);
+
+    // Make a cost based decision to pick cheaper plan
+    RelOptCost afterCost = RelMetadataQuery.getCumulativeCost(r);
+    RelOptCost beforeCost = RelMetadataQuery.getCumulativeCost(aggregate);
+    if (afterCost.isLt(beforeCost)) {
+      call.transformTo(r);
+    }
   }
 
   /** Computes the closure of a set of columns according to a given list of

http://git-wip-us.apache.org/repos/asf/hive/blob/a8e61c27/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q b/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
index b906db2..cbfb5d5 100644
--- a/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
+++ b/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
@@ -3,7 +3,7 @@ set hive.stats.fetch.column.stats=true;
 ;
 
 set hive.exec.reducers.max = 1;
-
+set hive.transpose.aggr.join=true;
 -- SORT_QUERY_RESULTS
 
 CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS;

http://git-wip-us.apache.org/repos/asf/hive/blob/a8e61c27/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
index 6537a8a..59a2f12 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
@@ -933,8 +933,10 @@ select count(*) from
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-2 depends on stages: Stage-1, Stage-4
+  Stage-3 depends on stages: Stage-2
+  Stage-4 is a root stage
+  Stage-0 depends on stages: Stage-3
 
 STAGE PLANS:
   Stage: Stage-1
@@ -947,41 +949,67 @@ STAGE PLANS:
               predicate: (key + 1) is not null (type: boolean)
               Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
-                expressions: key (type: int)
+                expressions: (key + 1) (type: int)
                 outputColumnNames: key
                 Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: (key + 1) (type: int)
-                  sort order: +
-                  Map-reduce partition columns: (key + 1) (type: int)
-                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  keys: key (type: int)
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key, $f1
+          Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
           TableScan
-            alias: subq2:a
-            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: (key + 1) is not null (type: boolean)
-              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: key (type: int)
-                outputColumnNames: key
-                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: (key + 1) (type: int)
-                  sort order: +
-                  Map-reduce partition columns: (key + 1) (type: int)
-                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Reduce Output Operator
+              key expressions: key (type: int)
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint)
+          TableScan
+            Reduce Output Operator
+              key expressions: key (type: int)
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 (key + 1) (type: int)
-            1 (key + 1) (type: int)
-          Statistics: Num rows: 5 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+            0 key (type: int)
+            1 key (type: int)
+          outputColumnNames: $f1, $f10
+          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            Statistics: Num rows: 5 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+            expressions: ($f1 * $f10) (type: bigint)
+            outputColumnNames: $f4
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
-              aggregations: count()
+              aggregations: $sum0($f4)
               mode: hash
               outputColumnNames: _col0
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -992,7 +1020,7 @@ STAGE PLANS:
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                     serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-2
+  Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -1002,7 +1030,7 @@ STAGE PLANS:
               value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
-          aggregations: count(VALUE._col0)
+          aggregations: $sum0(VALUE._col0)
           mode: mergepartial
           outputColumnNames: $f0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1014,6 +1042,45 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
+  Stage: Stage-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: subq2:a
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: (key + 1) is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: (key + 1) (type: int)
+                outputColumnNames: key
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  keys: key (type: int)
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key, $f1
+          Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
   Stage: Stage-0
     Fetch Operator
       limit: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/a8e61c27/ql/src/test/results/clientpositive/groupby_join_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_join_pushdown.q.out b/ql/src/test/results/clientpositive/groupby_join_pushdown.q.out
index 17df98f..c18e62f 100644
--- a/ql/src/test/results/clientpositive/groupby_join_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/groupby_join_pushdown.q.out
@@ -540,10 +540,8 @@ GROUP BY f.ctinyint, g.ctinyint
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1, Stage-4
-  Stage-3 depends on stages: Stage-2
-  Stage-4 is a root stage
-  Stage-0 depends on stages: Stage-3
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
 
 STAGE PLANS:
   Stage: Stage-1
@@ -559,49 +557,28 @@ STAGE PLANS:
                 expressions: ctinyint (type: tinyint), cint (type: int), cbigint (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: sum(_col2)
-                  keys: _col0 (type: tinyint), _col1 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
                   Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: tinyint), _col1 (type: int)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: int)
-                    Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: bigint)
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: sum(VALUE._col0)
-          keys: KEY._col0 (type: tinyint), KEY._col1 (type: int)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            table:
-                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col1 (type: int)
-              sort order: +
-              Map-reduce partition columns: _col1 (type: int)
-              Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
-              value expressions: _col0 (type: tinyint), _col2 (type: bigint)
+                  value expressions: _col0 (type: tinyint), _col2 (type: bigint)
           TableScan
-            Reduce Output Operator
-              key expressions: _col1 (type: int)
-              sort order: +
-              Map-reduce partition columns: _col1 (type: int)
-              Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
-              value expressions: _col0 (type: tinyint), _col2 (type: bigint)
+            alias: f
+            Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: cint is not null (type: boolean)
+              Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: ctinyint (type: tinyint), cint (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -609,18 +586,18 @@ STAGE PLANS:
           keys:
             0 _col1 (type: int)
             1 _col1 (type: int)
-          outputColumnNames: _col0, _col2, _col3, _col5
-          Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+          outputColumnNames: _col0, _col2, _col3
+          Statistics: Num rows: 6758 Data size: 1453080 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: tinyint), _col3 (type: tinyint), (_col2 * _col5) (type: bigint)
-            outputColumnNames: _col0, _col3, _col6
-            Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+            expressions: _col0 (type: tinyint), _col3 (type: tinyint), _col2 (type: bigint)
+            outputColumnNames: _col0, _col3, _col2
+            Statistics: Num rows: 6758 Data size: 1453080 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
-              aggregations: sum(_col6)
+              aggregations: sum(_col2)
               keys: _col0 (type: tinyint), _col3 (type: tinyint)
               mode: hash
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 6758 Data size: 1453080 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
                 table:
@@ -628,7 +605,7 @@ STAGE PLANS:
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                     serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-3
+  Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -636,7 +613,7 @@ STAGE PLANS:
               key expressions: _col0 (type: tinyint), _col1 (type: tinyint)
               sort order: ++
               Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: tinyint)
-              Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 6758 Data size: 1453080 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -644,54 +621,15 @@ STAGE PLANS:
           keys: KEY._col0 (type: tinyint), KEY._col1 (type: tinyint)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 3379 Data size: 726540 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-4
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: f
-            Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
-            Filter Operator
-              predicate: cint is not null (type: boolean)
-              Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: ctinyint (type: tinyint), cint (type: int)
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: tinyint), _col1 (type: int)
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: tinyint), _col1 (type: int)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: int)
-                    Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col2 (type: bigint)
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: count(VALUE._col0)
-          keys: KEY._col0 (type: tinyint), KEY._col1 (type: int)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            table:
-                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
   Stage: Stage-0
     Fetch Operator
       limit: -1