You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2019/09/27 17:28:24 UTC

[hive] branch master updated: HIVE-22163 : CBO: Enabling CBO turns on stats estimation, even when the estimation is disabled (Krisztian Kasa via Vineet Garg)

This is an automated email from the ASF dual-hosted git repository.

hashutosh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new b53521a  HIVE-22163 : CBO: Enabling CBO turns on stats estimation, even when the estimation is disabled (Krisztian Kasa via Vineet Garg)
b53521a is described below

commit b53521a1d4afabd99add9e3056dea03cae09ac1f
Author: Krisztian Kasa <kk...@cloudera.com>
AuthorDate: Fri Sep 27 10:27:08 2019 -0700

    HIVE-22163 : CBO: Enabling CBO turns on stats estimation, even when the estimation is disabled (Krisztian Kasa via Vineet Garg)
    
    Signed-off-by: Ashutosh Chauhan <ha...@apache.org>
---
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |  15 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java    |  11 +-
 .../queries/clientpositive/cbo_stats_estimation.q  |  11 +
 .../clientpositive/cbo_stats_estimation.q.out      | 278 +++++++++++++++++++++
 .../llap/join_reordering_no_stats.q.out            | 188 +++++++-------
 5 files changed, 401 insertions(+), 102 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index c3184a8..20ec058 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -1968,8 +1968,19 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // 10. Sort predicates in filter expressions
       if (conf.getBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_SORT_PREDS_WITH_STATS)) {
         perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
-        calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
-            HepMatchOrder.BOTTOM_UP, HiveFilterSortPredicates.INSTANCE);
+        try {
+          calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
+              HepMatchOrder.BOTTOM_UP, HiveFilterSortPredicates.INSTANCE);
+        } catch (Exception e) {
+          boolean isMissingStats = noColsMissingStats.get() > 0;
+          if (isMissingStats) {
+            LOG.warn("Missing column stats (see previous messages), " +
+                    "skipping sort predicates in filter expressions in CBO");
+            noColsMissingStats.set(0);
+          } else {
+            throw e;
+          }
+        }
         perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER,
             "Calcite: Sort predicates within filter operators");
       }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index 1795ae5..9a00a75 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -269,11 +269,14 @@ public class StatsUtils {
 
       if (needColStats) {
         colStats = getTableColumnStats(table, schema, neededColumns, colStatsCache, fetchColStats);
-        estimateStatsForMissingCols(neededColumns, colStats, table, conf, nr, schema);
+        if (estimateStats) {
+          estimateStatsForMissingCols(neededColumns, colStats, table, conf, nr, schema);
+        }
         // we should have stats for all columns (estimated or actual)
-        assert (neededColumns.size() == colStats.size());
-        long betterDS = getDataSizeFromColumnStats(nr, colStats);
-        ds = (betterDS < 1 || colStats.isEmpty()) ? ds : betterDS;
+        if (neededColumns.size() == colStats.size()) {
+          long betterDS = getDataSizeFromColumnStats(nr, colStats);
+          ds = (betterDS < 1 || colStats.isEmpty()) ? ds : betterDS;
+        }
       }
 
       stats = new Statistics(nr, ds, numErasureCodedFiles);
diff --git a/ql/src/test/queries/clientpositive/cbo_stats_estimation.q b/ql/src/test/queries/clientpositive/cbo_stats_estimation.q
new file mode 100644
index 0000000..80908f8
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/cbo_stats_estimation.q
@@ -0,0 +1,11 @@
+CREATE TABLE claims(claim_rec_id bigint, claim_invoice_num string, typ_c int);
+ALTER TABLE claims UPDATE STATISTICS set ('numRows'='1154941534','rawDataSize'='1135307527922');
+
+
+SET hive.stats.estimate=false;
+
+EXPLAIN EXTENDED SELECT count(1) FROM claims WHERE typ_c=3;
+
+SET hive.stats.ndv.estimate.percent=5e-7;
+
+EXPLAIN EXTENDED SELECT count(1) FROM claims WHERE typ_c=3;
diff --git a/ql/src/test/results/clientpositive/cbo_stats_estimation.q.out b/ql/src/test/results/clientpositive/cbo_stats_estimation.q.out
new file mode 100644
index 0000000..389a9bc
--- /dev/null
+++ b/ql/src/test/results/clientpositive/cbo_stats_estimation.q.out
@@ -0,0 +1,278 @@
+PREHOOK: query: CREATE TABLE claims(claim_rec_id bigint, claim_invoice_num string, typ_c int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@claims
+POSTHOOK: query: CREATE TABLE claims(claim_rec_id bigint, claim_invoice_num string, typ_c int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@claims
+PREHOOK: query: ALTER TABLE claims UPDATE STATISTICS set ('numRows'='1154941534','rawDataSize'='1135307527922')
+PREHOOK: type: ALTERTABLE_UPDATETABLESTATS
+PREHOOK: Input: default@claims
+PREHOOK: Output: default@claims
+POSTHOOK: query: ALTER TABLE claims UPDATE STATISTICS set ('numRows'='1154941534','rawDataSize'='1135307527922')
+POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS
+POSTHOOK: Input: default@claims
+POSTHOOK: Output: default@claims
+PREHOOK: query: EXPLAIN EXTENDED SELECT count(1) FROM claims WHERE typ_c=3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@claims
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN EXTENDED SELECT count(1) FROM claims WHERE typ_c=3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@claims
+#### A masked pattern was here ####
+OPTIMIZED SQL: SELECT COUNT(*) AS `$f0`
+FROM `default`.`claims`
+WHERE `typ_c` = 3
+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: claims
+            filterExpr: (typ_c = 3) (type: boolean)
+            Statistics: Num rows: 1154941534 Data size: 1135307527922 Basic stats: COMPLETE Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (typ_c = 3) (type: boolean)
+              Statistics: Num rows: 577470767 Data size: 567653763961 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                Statistics: Num rows: 577470767 Data size: 567653763961 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    tag: -1
+                    value expressions: _col0 (type: bigint)
+                    auto parallelism: false
+      Execution mode: vectorized
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: claims
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              bucketing_version 2
+              column.name.delimiter ,
+              columns claim_rec_id,claim_invoice_num,typ_c
+              columns.comments 
+              columns.types bigint:string:int
+#### A masked pattern was here ####
+              name default.claims
+              numFiles 0
+              numRows 1154941534
+              rawDataSize 1135307527922
+              serialization.ddl struct claims { i64 claim_rec_id, string claim_invoice_num, i32 typ_c}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 0
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                bucketing_version 2
+                column.name.delimiter ,
+                columns claim_rec_id,claim_invoice_num,typ_c
+                columns.comments 
+                columns.types bigint:string:int
+#### A masked pattern was here ####
+                name default.claims
+                numFiles 0
+                numRows 1154941534
+                rawDataSize 1135307527922
+                serialization.ddl struct claims { i64 claim_rec_id, string claim_invoice_num, i32 typ_c}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 0
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.claims
+            name: default.claims
+      Truncated Path -> Alias:
+        /claims [claims]
+      Needs Tagging: false
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            GlobalTableId: 0
+#### A masked pattern was here ####
+            NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+#### A masked pattern was here ####
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                properties:
+                  columns _col0
+                  columns.types bigint
+                  escape.delim \
+                  hive.serialization.extend.additional.nesting.levels true
+                  serialization.escape.crlf true
+                  serialization.format 1
+                  serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            TotalFiles: 1
+            GatherStats: false
+            MultiFileSpray: false
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN EXTENDED SELECT count(1) FROM claims WHERE typ_c=3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@claims
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN EXTENDED SELECT count(1) FROM claims WHERE typ_c=3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@claims
+#### A masked pattern was here ####
+OPTIMIZED SQL: SELECT COUNT(*) AS `$f0`
+FROM `default`.`claims`
+WHERE `typ_c` = 3
+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: claims
+            filterExpr: (typ_c = 3) (type: boolean)
+            Statistics: Num rows: 1154941534 Data size: 1135307527922 Basic stats: COMPLETE Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (typ_c = 3) (type: boolean)
+              Statistics: Num rows: 577470767 Data size: 567653763961 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                Statistics: Num rows: 577470767 Data size: 567653763961 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    tag: -1
+                    value expressions: _col0 (type: bigint)
+                    auto parallelism: false
+      Execution mode: vectorized
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: claims
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              bucketing_version 2
+              column.name.delimiter ,
+              columns claim_rec_id,claim_invoice_num,typ_c
+              columns.comments 
+              columns.types bigint:string:int
+#### A masked pattern was here ####
+              name default.claims
+              numFiles 0
+              numRows 1154941534
+              rawDataSize 1135307527922
+              serialization.ddl struct claims { i64 claim_rec_id, string claim_invoice_num, i32 typ_c}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 0
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                bucketing_version 2
+                column.name.delimiter ,
+                columns claim_rec_id,claim_invoice_num,typ_c
+                columns.comments 
+                columns.types bigint:string:int
+#### A masked pattern was here ####
+                name default.claims
+                numFiles 0
+                numRows 1154941534
+                rawDataSize 1135307527922
+                serialization.ddl struct claims { i64 claim_rec_id, string claim_invoice_num, i32 typ_c}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 0
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.claims
+            name: default.claims
+      Truncated Path -> Alias:
+        /claims [claims]
+      Needs Tagging: false
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            GlobalTableId: 0
+#### A masked pattern was here ####
+            NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+#### A masked pattern was here ####
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                properties:
+                  columns _col0
+                  columns.types bigint
+                  escape.delim \
+                  hive.serialization.extend.additional.nesting.levels true
+                  serialization.escape.crlf true
+                  serialization.format 1
+                  serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            TotalFiles: 1
+            GatherStats: false
+            MultiFileSpray: false
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
diff --git a/ql/src/test/results/clientpositive/llap/join_reordering_no_stats.q.out b/ql/src/test/results/clientpositive/llap/join_reordering_no_stats.q.out
index fddffbb..df15d59 100644
--- a/ql/src/test/results/clientpositive/llap/join_reordering_no_stats.q.out
+++ b/ql/src/test/results/clientpositive/llap/join_reordering_no_stats.q.out
@@ -223,6 +223,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain select count(1) from part_nostats,supplier_nostats,lineitem_nostats where p_partkey = l_partkey and s_suppkey = l_suppkey
 PREHOOK: type: QUERY
 PREHOOK: Input: default@lineitem_nostats
@@ -244,7 +245,7 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (XPROD_EDGE), Map 5 (XPROD_EDGE)
         Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -252,62 +253,59 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: lineitem_nostats
-                  filterExpr: (l_partkey is not null and l_suppkey is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (l_partkey is not null and l_suppkey is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: l_partkey (type: int), l_suppkey (type: int)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
                   alias: part_nostats
                   filterExpr: p_partkey is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: -1 Basic stats: PARTIAL Column stats: NONE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: p_partkey (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 6 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: supplier_nostats
                   filterExpr: s_suppkey is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: -1 Basic stats: PARTIAL Column stats: NONE
                   Filter Operator
                     predicate: s_suppkey is not null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: s_suppkey (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: lineitem_nostats
+                  filterExpr: (l_partkey is not null and l_suppkey is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: -1 Basic stats: PARTIAL Column stats: NONE
+                  Filter Operator
+                    predicate: (l_partkey is not null and l_suppkey is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: l_partkey (type: int), l_suppkey (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: int)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                        Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -317,15 +315,15 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col1
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col1 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col1 (type: int)
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  key expressions: _col0 (type: int), _col1 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                  Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -333,18 +331,18 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: int)
-                  1 _col0 (type: int)
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  0 _col0 (type: int), _col1 (type: int)
+                  1 _col0 (type: int), _col1 (type: int)
+                Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
                     sort order: 
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     value expressions: _col0 (type: bigint)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -353,10 +351,10 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -589,6 +587,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain select count(1) from Employee_Part_n1,supplier_nostats,lineitem_nostats where employeeID= l_partkey and s_suppkey = l_suppkey
 PREHOOK: type: QUERY
 PREHOOK: Input: default@employee_part_n1
@@ -622,7 +621,7 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (XPROD_EDGE), Map 5 (XPROD_EDGE)
         Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -630,27 +629,6 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: lineitem_nostats
-                  filterExpr: (l_partkey is not null and l_suppkey is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (l_partkey is not null and l_suppkey is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: l_partkey (type: int), l_suppkey (type: int)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
                   alias: employee_part_n1
                   filterExpr: employeeid is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: PARTIAL Column stats: NONE
@@ -662,30 +640,48 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                        sort order: 
                         Statistics: Num rows: 1 Data size: 4 Basic stats: PARTIAL Column stats: NONE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 6 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: supplier_nostats
                   filterExpr: s_suppkey is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: -1 Basic stats: PARTIAL Column stats: NONE
                   Filter Operator
                     predicate: s_suppkey is not null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: s_suppkey (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: lineitem_nostats
+                  filterExpr: (l_partkey is not null and l_suppkey is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: -1 Basic stats: PARTIAL Column stats: NONE
+                  Filter Operator
+                    predicate: (l_partkey is not null and l_suppkey is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: l_partkey (type: int), l_suppkey (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: int)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                        Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -695,15 +691,15 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col1
-                Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col1 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col1 (type: int)
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
+                  key expressions: _col0 (type: int), _col1 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                  Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -711,9 +707,9 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: int)
-                  1 _col0 (type: int)
-                Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
+                  0 _col0 (type: int), _col1 (type: int)
+                  1 _col0 (type: int), _col1 (type: int)
+                Statistics: Num rows: 1 Data size: 9223372036854775807 Basic stats: PARTIAL Column stats: NONE
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99