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 82e975a50d..5c4f611704 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 @@ -1796,16 +1796,17 @@ public RelNode apply(RelOptCluster cluster, RelOptSchema relOptSchema, SchemaPlu perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Removing sq_count_check UDF "); } - // Remove Projects between Joins so that JoinToMultiJoinRule can merge them to MultiJoin - calcitePreCboPlan = hepPlan(calcitePreCboPlan, true, mdProvider.getMetadataProvider(), executorProvider, - HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.LEFT_PROJECT_BTW_JOIN, - HiveJoinProjectTransposeRule.RIGHT_PROJECT_BTW_JOIN, HiveProjectMergeRule.INSTANCE); // 4. Apply join order optimizations: reordering MST algorithm // If join optimizations failed because of missing stats, we continue with // the rest of optimizations if (profilesCBO.contains(ExtendedCBOProfile.JOIN_REORDERING)) { perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER); + + // Remove Projects between Joins so that JoinToMultiJoinRule can merge them to MultiJoin + calcitePreCboPlan = hepPlan(calcitePreCboPlan, true, mdProvider.getMetadataProvider(), executorProvider, + HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.LEFT_PROJECT_BTW_JOIN, + HiveJoinProjectTransposeRule.RIGHT_PROJECT_BTW_JOIN, HiveProjectMergeRule.INSTANCE); try { List list = Lists.newArrayList(); list.add(mdProvider.getMetadataProvider()); @@ -3343,6 +3344,10 @@ private boolean genSubQueryRelNode(QB qb, ASTNode node, RelNode srcRel, boolean } catch (SemanticException e) { throw new CalciteSubquerySemanticException(e.getMessage()); } + if(isSubQuery) { + // since subqueries will later be rewritten into JOINs we want join reordering logic to trigger + profilesCBO.add(ExtendedCBOProfile.JOIN_REORDERING); + } return isSubQuery; } diff --git a/ql/src/test/results/clientpositive/constant_prop_3.q.out b/ql/src/test/results/clientpositive/constant_prop_3.q.out index d106cd325a..2b314d7ebd 100644 --- a/ql/src/test/results/clientpositive/constant_prop_3.q.out +++ b/ql/src/test/results/clientpositive/constant_prop_3.q.out @@ -94,7 +94,7 @@ POSTHOOK: type: ANALYZE_TABLE POSTHOOK: Input: default@supplier_hive POSTHOOK: Output: default@supplier_hive #### A masked pattern was here #### -Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain select p_brand, p_type, @@ -166,36 +166,56 @@ POSTHOOK: Input: default@partsupp_hive POSTHOOK: Input: default@supplier_hive #### A masked pattern was here #### STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1, Stage-7 - Stage-3 depends on stages: Stage-2, Stage-8 + Stage-5 is a root stage + Stage-1 depends on stages: Stage-5 + Stage-2 depends on stages: Stage-1, Stage-6 + Stage-3 depends on stages: Stage-2 Stage-4 depends on stages: Stage-3 - Stage-5 depends on stages: Stage-4 Stage-7 is a root stage - Stage-8 is a root stage - Stage-0 depends on stages: Stage-5 + Stage-6 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-4 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-5 Map Reduce Map Operator Tree: TableScan - alias: partsupp_hive - filterExpr: ps_partkey is not null (type: boolean) + alias: supplier_hive + filterExpr: (s_comment like '%Customer%Complaints%') (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: ps_partkey is not null (type: boolean) + predicate: (s_comment like '%Customer%Complaints%') (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: ps_partkey (type: int), ps_suppkey (type: int) - outputColumnNames: _col0, _col1 + expressions: s_suppkey (type: int) + outputColumnNames: s_suppkey Statistics: Num rows: 1 Data size: 0 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: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col1 (type: int) + Group By Operator + aggregations: count(), count(s_suppkey) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: PARTIAL Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: PARTIAL 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-1 + Map Reduce + Map Operator Tree: TableScan alias: part_hive filterExpr: ((p_size) IN (22, 14, 27, 49, 21, 33, 35, 28) and (p_brand <> 'Brand#34') and p_partkey is not null and (not (p_type like 'ECONOMY BRUSHED%'))) (type: boolean) @@ -208,35 +228,9 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 0 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: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int) - Reduce Operator Tree: - Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col1, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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 - sort order: - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col1 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int) + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: int) TableScan Reduce Output Operator sort order: @@ -249,7 +243,7 @@ STAGE PLANS: keys: 0 1 - outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 1 Data size: 17 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: false @@ -258,53 +252,57 @@ 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 Reduce Output Operator - key expressions: _col1 (type: int) + key expressions: _col0 (type: int) sort order: + - Map-reduce partition columns: _col1 (type: int) + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 17 Basic stats: PARTIAL Column stats: NONE - value expressions: _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: bigint), _col7 (type: bigint) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: bigint) TableScan Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col1 (type: boolean) + value expressions: _col1 (type: int), _col3 (type: boolean) Reduce Operator Tree: Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col1 (type: int) + 0 _col0 (type: int) 1 _col0 (type: int) - outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col9 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col7, _col9 Statistics: Num rows: 1 Data size: 18 Basic stats: PARTIAL Column stats: NONE - Filter Operator - predicate: ((_col6 = 0L) or (_col9 is null and _col1 is not null and (_col7 >= _col6))) (type: boolean) + Select Operator + expressions: _col7 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col9 (type: boolean) + outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col9 Statistics: Num rows: 1 Data size: 18 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: _col1 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int) - outputColumnNames: _col1, _col3, _col4, _col5 + Filter Operator + predicate: ((_col6 = 0L) or (_col9 is null and _col1 is not null and (_col7 >= _col6))) (type: boolean) Statistics: Num rows: 1 Data size: 18 Basic stats: PARTIAL Column stats: NONE - Group By Operator - aggregations: count(DISTINCT _col1) - keys: _col3 (type: string), _col4 (type: string), _col5 (type: int), _col1 (type: int) - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: _col1 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int) + outputColumnNames: _col1, _col3, _col4, _col5 Statistics: Num rows: 1 Data size: 18 Basic stats: PARTIAL 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 + Group By Operator + aggregations: count(DISTINCT _col1) + keys: _col3 (type: string), _col4 (type: string), _col5 (type: int), _col1 (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 18 Basic stats: PARTIAL 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-4 + Stage: Stage-3 Map Reduce Map Operator Tree: TableScan @@ -328,7 +326,7 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-5 + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan @@ -351,43 +349,6 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Stage: Stage-7 - Map Reduce - Map Operator Tree: - TableScan - alias: supplier_hive - filterExpr: (s_comment like '%Customer%Complaints%') (type: boolean) - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Filter Operator - predicate: (s_comment like '%Customer%Complaints%') (type: boolean) - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: s_suppkey (type: int) - outputColumnNames: s_suppkey - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Group By Operator - aggregations: count(), count(s_suppkey) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: PARTIAL Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: PARTIAL Column stats: NONE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: PARTIAL 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-8 Map Reduce Map Operator Tree: TableScan @@ -429,6 +390,49 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Stage: Stage-6 + Map Reduce + Map Operator Tree: + TableScan + alias: partsupp_hive + filterExpr: ps_partkey is not null (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: ps_partkey is not null (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: ps_partkey (type: int), ps_suppkey (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 0 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: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col0 (type: int) + TableScan + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: boolean) + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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 diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out index 801416d563..2810fdd1a2 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out @@ -80,10 +80,10 @@ STAGE PLANS: #### A masked pattern was here #### Edges: Map 1 <- Reducer 11 (BROADCAST_EDGE), Reducer 6 (BROADCAST_EDGE), Reducer 7 (BROADCAST_EDGE), Reducer 8 (BROADCAST_EDGE) - Map 13 <- Reducer 12 (BROADCAST_EDGE) - Reducer 10 <- Map 13 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE) + Map 9 <- Reducer 13 (BROADCAST_EDGE) + Reducer 10 <- Map 12 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE) Reducer 11 <- Reducer 10 (CUSTOM_SIMPLE_EDGE) - Reducer 12 <- Map 9 (CUSTOM_SIMPLE_EDGE) + Reducer 13 <- Map 12 (CUSTOM_SIMPLE_EDGE) Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) Reducer 3 <- Reducer 10 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE) @@ -113,25 +113,37 @@ STAGE PLANS: value expressions: _col3 (type: timestamp) Execution mode: vectorized, llap LLAP IO: no inputs - Map 13 + Map 12 Map Operator Tree: TableScan - alias: tt2 - filterExpr: (timestamp_col_18 is not null and decimal1911_col_16 is not null and (decimal1911_col_16 BETWEEN DynamicValue(RS_12_tt1_decimal2612_col_77_min) AND DynamicValue(RS_12_tt1_decimal2612_col_77_max) and in_bloom_filter(decimal1911_col_16, DynamicValue(RS_12_tt1_decimal2612_col_77_bloom_filter)))) (type: boolean) - Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE + alias: tt1 + filterExpr: decimal2612_col_77 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((decimal1911_col_16 BETWEEN DynamicValue(RS_12_tt1_decimal2612_col_77_min) AND DynamicValue(RS_12_tt1_decimal2612_col_77_max) and in_bloom_filter(decimal1911_col_16, DynamicValue(RS_12_tt1_decimal2612_col_77_bloom_filter))) and decimal1911_col_16 is not null and timestamp_col_18 is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE + predicate: decimal2612_col_77 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: decimal1911_col_16 (type: decimal(19,11)), timestamp_col_18 (type: timestamp) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE + expressions: decimal2612_col_77 (type: decimal(26,12)) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: decimal(26,12)) sort order: + Map-reduce partition columns: _col0 (type: decimal(26,12)) - Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: timestamp) + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: decimal(26,12)) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 448 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 448 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: decimal(26,12)), _col1 (type: decimal(26,12)), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: no inputs Map 5 @@ -197,34 +209,22 @@ STAGE PLANS: Map 9 Map Operator Tree: TableScan - alias: tt1 - filterExpr: decimal2612_col_77 is not null (type: boolean) - Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE + alias: tt2 + filterExpr: (timestamp_col_18 is not null and decimal1911_col_16 is not null and (decimal1911_col_16 BETWEEN DynamicValue(RS_13_tt1_decimal2612_col_77_min) AND DynamicValue(RS_13_tt1_decimal2612_col_77_max) and in_bloom_filter(decimal1911_col_16, DynamicValue(RS_13_tt1_decimal2612_col_77_bloom_filter)))) (type: boolean) + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: decimal2612_col_77 is not null (type: boolean) - Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE + predicate: ((decimal1911_col_16 BETWEEN DynamicValue(RS_13_tt1_decimal2612_col_77_min) AND DynamicValue(RS_13_tt1_decimal2612_col_77_max) and in_bloom_filter(decimal1911_col_16, DynamicValue(RS_13_tt1_decimal2612_col_77_bloom_filter))) and decimal1911_col_16 is not null and timestamp_col_18 is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: decimal2612_col_77 (type: decimal(26,12)) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE + expressions: decimal1911_col_16 (type: decimal(19,11)), timestamp_col_18 (type: timestamp) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: decimal(26,12)) sort order: + Map-reduce partition columns: _col0 (type: decimal(26,12)) - Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: decimal(26,12)) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE - Group By Operator - aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1) - mode: hash - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 448 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 448 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: decimal(26,12)), _col1 (type: decimal(26,12)), _col2 (type: binary) + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: timestamp) Execution mode: vectorized, llap LLAP IO: no inputs Reducer 10 @@ -236,26 +236,26 @@ STAGE PLANS: keys: 0 _col0 (type: decimal(26,12)) 1 _col0 (type: decimal(26,12)) - outputColumnNames: _col2 - Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 167 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col2 (type: timestamp), -92 (type: int) + expressions: _col1 (type: timestamp), -92 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 167 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: timestamp), _col1 (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 167 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: timestamp), _col1 (type: int) sort order: ++ Map-reduce partition columns: _col0 (type: timestamp), _col1 (type: int) - Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 167 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: timestamp) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 167 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1) mode: hash @@ -277,7 +277,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary) - Reducer 12 + Reducer 13 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out index f240468558..ed2bfe1590 100644 --- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out +++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out @@ -2251,36 +2251,42 @@ POSTHOOK: Input: default@src_cbo Plan optimized by CBO. Vertex dependency in root stage -Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE) +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) Stage-0 Fetch Operator limit:-1 Stage-1 - Reducer 2 llap - File Output Operator [FS_12] - Merge Join Operator [MERGEJOIN_27] (rows=83 width=178) - Conds:RS_8._col0=RS_9._col0(Left Semi),Output:["_col0","_col1"] - <-Map 1 [SIMPLE_EDGE] llap - SHUFFLE [RS_8] - PartitionCols:_col0 - Select Operator [SEL_2] (rows=166 width=178) - Output:["_col0","_col1"] - Filter Operator [FIL_15] (rows=166 width=178) - predicate:(key > '9') - TableScan [TS_0] (rows=500 width=178) - default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] - <-Map 3 [SIMPLE_EDGE] llap - SHUFFLE [RS_9] - PartitionCols:_col0 - Group By Operator [GBY_7] (rows=83 width=87) - Output:["_col0"],keys:_col0 - Select Operator [SEL_5] (rows=166 width=87) - Output:["_col0"] - Filter Operator [FIL_16] (rows=166 width=87) + Reducer 3 llap + File Output Operator [FS_14] + Select Operator [SEL_13] (rows=83 width=178) + Output:["_col0","_col1"] + Merge Join Operator [MERGEJOIN_29] (rows=83 width=178) + Conds:RS_10._col0=RS_11._col0(Inner),Output:["_col1","_col2"] + <-Map 4 [SIMPLE_EDGE] llap + SHUFFLE [RS_11] + PartitionCols:_col0 + Select Operator [SEL_9] (rows=166 width=178) + Output:["_col0","_col1"] + Filter Operator [FIL_18] (rows=166 width=178) predicate:(key > '9') - TableScan [TS_3] (rows=500 width=87) - default@src_cbo,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] + TableScan [TS_7] (rows=500 width=178) + default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] + <-Reducer 2 [ONE_TO_ONE_EDGE] llap + FORWARD [RS_10] + PartitionCols:_col0 + Group By Operator [GBY_5] (rows=83 width=87) + Output:["_col0"],keys:KEY._col0 + <-Map 1 [SIMPLE_EDGE] llap + SHUFFLE [RS_4] + PartitionCols:_col0 + Group By Operator [GBY_3] (rows=83 width=87) + Output:["_col0"],keys:key + Filter Operator [FIL_17] (rows=166 width=87) + predicate:(key > '9') + TableScan [TS_0] (rows=500 width=87) + default@src_cbo,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] PREHOOK: query: explain select p.p_partkey, li.l_suppkey from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey @@ -2299,58 +2305,56 @@ POSTHOOK: Input: default@lineitem Plan optimized by CBO. Vertex dependency in root stage -Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) -Reducer 4 <- Map 6 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) +Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) +Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) Stage-0 Fetch Operator limit:-1 Stage-1 - Reducer 4 llap - File Output Operator [FS_22] - Select Operator [SEL_21] (rows=1 width=8) + Reducer 3 llap + File Output Operator [FS_21] + Select Operator [SEL_20] (rows=1 width=8) Output:["_col0","_col1"] - Merge Join Operator [MERGEJOIN_52] (rows=1 width=8) - Conds:RS_18._col1, _col4=RS_19._col0, _col1(Left Semi),Output:["_col0","_col3"] - <-Map 6 [SIMPLE_EDGE] llap - SHUFFLE [RS_19] + Merge Join Operator [MERGEJOIN_51] (rows=1 width=8) + Conds:RS_17._col1, _col4=RS_18._col0, _col1(Left Semi),Output:["_col0","_col3"] + <-Map 5 [SIMPLE_EDGE] llap + SHUFFLE [RS_18] PartitionCols:_col0, _col1 - Group By Operator [GBY_17] (rows=1 width=8) + Group By Operator [GBY_16] (rows=1 width=8) Output:["_col0","_col1"],keys:_col0, _col1 - Select Operator [SEL_12] (rows=2 width=8) + Select Operator [SEL_14] (rows=2 width=8) Output:["_col0","_col1"] - Filter Operator [FIL_30] (rows=2 width=96) + Filter Operator [FIL_29] (rows=2 width=96) predicate:((l_linenumber = 1) and (l_shipmode = 'AIR') and l_orderkey is not null) - TableScan [TS_10] (rows=100 width=96) + TableScan [TS_12] (rows=100 width=96) default@lineitem,lineitem,Tbl:COMPLETE,Col:COMPLETE,Output:["l_orderkey","l_linenumber","l_shipmode"] - <-Reducer 3 [SIMPLE_EDGE] llap - SHUFFLE [RS_18] + <-Reducer 2 [SIMPLE_EDGE] llap + SHUFFLE [RS_17] PartitionCols:_col1, _col4 - Merge Join Operator [MERGEJOIN_51] (rows=14 width=16) - Conds:RS_13._col0=RS_14._col1(Inner),Output:["_col0","_col1","_col3","_col4"] - <-Map 5 [SIMPLE_EDGE] llap - SHUFFLE [RS_14] - PartitionCols:_col1 - Select Operator [SEL_9] (rows=14 width=16) - Output:["_col0","_col1","_col2","_col3"] - Filter Operator [FIL_29] (rows=14 width=16) - predicate:((l_linenumber = 1) and l_orderkey is not null and l_partkey is not null) - TableScan [TS_7] (rows=100 width=16) - default@lineitem,li,Tbl:COMPLETE,Col:COMPLETE,Output:["l_orderkey","l_partkey","l_suppkey","l_linenumber"] - <-Reducer 2 [ONE_TO_ONE_EDGE] llap - FORWARD [RS_13] - PartitionCols:_col0 - Group By Operator [GBY_5] (rows=50 width=4) - Output:["_col0"],keys:KEY._col0 - <-Map 1 [SIMPLE_EDGE] llap - SHUFFLE [RS_4] - PartitionCols:_col0 - Group By Operator [GBY_3] (rows=50 width=4) - Output:["_col0"],keys:l_partkey + Select Operator [SEL_11] (rows=14 width=16) + Output:["_col0","_col1","_col3","_col4"] + Merge Join Operator [MERGEJOIN_50] (rows=14 width=16) + Conds:RS_8._col1=RS_9._col0(Left Semi),Output:["_col0","_col1","_col2","_col3"] + <-Map 1 [SIMPLE_EDGE] llap + SHUFFLE [RS_8] + PartitionCols:_col1 + Select Operator [SEL_2] (rows=14 width=16) + Output:["_col0","_col1","_col2","_col3"] + Filter Operator [FIL_27] (rows=14 width=16) + predicate:((l_linenumber = 1) and l_orderkey is not null and l_partkey is not null) + TableScan [TS_0] (rows=100 width=16) + default@lineitem,li,Tbl:COMPLETE,Col:COMPLETE,Output:["l_orderkey","l_partkey","l_suppkey","l_linenumber"] + <-Map 4 [SIMPLE_EDGE] llap + SHUFFLE [RS_9] + PartitionCols:_col0 + Group By Operator [GBY_7] (rows=50 width=4) + Output:["_col0"],keys:_col0 + Select Operator [SEL_5] (rows=100 width=4) + Output:["_col0"] Filter Operator [FIL_28] (rows=100 width=4) predicate:l_partkey is not null - TableScan [TS_0] (rows=100 width=4) + TableScan [TS_3] (rows=100 width=4) default@lineitem,lineitem,Tbl:COMPLETE,Col:COMPLETE,Output:["l_partkey"] PREHOOK: query: explain select key, value, count(*) @@ -2372,74 +2376,78 @@ POSTHOOK: Input: default@src_cbo Plan optimized by CBO. Vertex dependency in root stage -Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) -Reducer 3 <- Reducer 2 (SIMPLE_EDGE) -Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) -Reducer 6 <- Map 5 (SIMPLE_EDGE) +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) +Reducer 4 <- Reducer 3 (SIMPLE_EDGE) +Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) +Reducer 6 <- Map 1 (SIMPLE_EDGE) Stage-0 Fetch Operator limit:-1 Stage-1 - Reducer 4 llap - File Output Operator [FS_31] - Merge Join Operator [MERGEJOIN_54] (rows=41 width=186) - Conds:RS_27._col2=RS_28._col0(Left Semi),Output:["_col0","_col1","_col2"] - <-Reducer 3 [SIMPLE_EDGE] llap - SHUFFLE [RS_27] + Reducer 5 llap + File Output Operator [FS_33] + Merge Join Operator [MERGEJOIN_56] (rows=41 width=186) + Conds:RS_29._col2=RS_30._col0(Left Semi),Output:["_col0","_col1","_col2"] + <-Reducer 4 [SIMPLE_EDGE] llap + SHUFFLE [RS_29] PartitionCols:_col2 - Filter Operator [FIL_37] (rows=41 width=186) + Filter Operator [FIL_39] (rows=41 width=186) predicate:_col2 is not null - Group By Operator [GBY_14] (rows=41 width=186) + Group By Operator [GBY_16] (rows=41 width=186) Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1 - <-Reducer 2 [SIMPLE_EDGE] llap - SHUFFLE [RS_13] + <-Reducer 3 [SIMPLE_EDGE] llap + SHUFFLE [RS_15] PartitionCols:_col0, _col1 - Group By Operator [GBY_12] (rows=41 width=186) - Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col0, _col1 - Merge Join Operator [MERGEJOIN_53] (rows=83 width=178) - Conds:RS_8._col0=RS_9._col0(Left Semi),Output:["_col0","_col1"] - <-Map 5 [SIMPLE_EDGE] llap - SHUFFLE [RS_9] + Group By Operator [GBY_14] (rows=41 width=186) + Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col1, _col2 + Merge Join Operator [MERGEJOIN_55] (rows=83 width=178) + Conds:RS_10._col0=RS_11._col0(Inner),Output:["_col1","_col2"] + <-Map 7 [SIMPLE_EDGE] llap + SHUFFLE [RS_11] PartitionCols:_col0 - Group By Operator [GBY_7] (rows=83 width=87) - Output:["_col0"],keys:_col0 - Select Operator [SEL_5] (rows=166 width=87) - Output:["_col0"] - Filter Operator [FIL_39] (rows=166 width=87) - predicate:(key > '8') - TableScan [TS_3] (rows=500 width=87) - default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] - <-Map 1 [SIMPLE_EDGE] llap - SHUFFLE [RS_8] - PartitionCols:_col0 - Select Operator [SEL_2] (rows=166 width=178) + Select Operator [SEL_9] (rows=166 width=178) Output:["_col0","_col1"] - Filter Operator [FIL_38] (rows=166 width=178) + Filter Operator [FIL_41] (rows=166 width=178) predicate:(key > '8') - TableScan [TS_0] (rows=500 width=178) + TableScan [TS_7] (rows=500 width=178) default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] + <-Reducer 2 [ONE_TO_ONE_EDGE] llap + FORWARD [RS_10] + PartitionCols:_col0 + Group By Operator [GBY_5] (rows=83 width=87) + Output:["_col0"],keys:KEY._col0 + <-Map 1 [SIMPLE_EDGE] llap + SHUFFLE [RS_4] + PartitionCols:_col0 + Group By Operator [GBY_3] (rows=83 width=87) + Output:["_col0"],keys:key + Filter Operator [FIL_40] (rows=166 width=87) + predicate:(key > '8') + TableScan [TS_0] (rows=500 width=87) + default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] <-Reducer 6 [SIMPLE_EDGE] llap - SHUFFLE [RS_28] + SHUFFLE [RS_30] PartitionCols:_col0 - Group By Operator [GBY_26] (rows=41 width=8) + Group By Operator [GBY_28] (rows=41 width=8) Output:["_col0"],keys:_col0 - Select Operator [SEL_24] (rows=83 width=8) + Select Operator [SEL_26] (rows=83 width=8) Output:["_col0"] - Filter Operator [FIL_40] (rows=83 width=8) + Filter Operator [FIL_42] (rows=83 width=8) predicate:_col1 is not null - Select Operator [SEL_42] (rows=83 width=8) + Select Operator [SEL_44] (rows=83 width=8) Output:["_col1"] - Group By Operator [GBY_22] (rows=83 width=95) + Group By Operator [GBY_24] (rows=83 width=95) Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0 - <-Map 5 [SIMPLE_EDGE] llap - SHUFFLE [RS_21] + <-Map 1 [SIMPLE_EDGE] llap + SHUFFLE [RS_23] PartitionCols:_col0 - Group By Operator [GBY_20] (rows=83 width=95) + Group By Operator [GBY_22] (rows=83 width=95) Output:["_col0","_col1"],aggregations:["count()"],keys:key - Filter Operator [FIL_41] (rows=166 width=87) + Filter Operator [FIL_43] (rows=166 width=87) predicate:(key > '9') - Please refer to the previous TableScan [TS_3] + Please refer to the previous TableScan [TS_0] PREHOOK: query: explain select p_mfgr, p_name, avg(p_size) from part @@ -2528,64 +2536,66 @@ POSTHOOK: Input: default@src_cbo Plan optimized by CBO. Vertex dependency in root stage -Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE) -Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) +Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) +Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE) Reducer 4 <- Reducer 3 (SIMPLE_EDGE) -Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE) -Reducer 7 <- Map 5 (SIMPLE_EDGE) +Reducer 6 <- Map 5 (SIMPLE_EDGE) +Reducer 7 <- Map 5 (CUSTOM_SIMPLE_EDGE) Stage-0 Fetch Operator limit:-1 Stage-1 Reducer 4 llap - File Output Operator [FS_26] - Select Operator [SEL_25] (rows=631 width=178) + File Output Operator [FS_27] + Select Operator [SEL_26] (rows=631 width=178) Output:["_col0","_col1"] <-Reducer 3 [SIMPLE_EDGE] llap - SHUFFLE [RS_24] - Select Operator [SEL_23] (rows=631 width=178) + SHUFFLE [RS_25] + Select Operator [SEL_24] (rows=631 width=178) Output:["_col0","_col1"] - Filter Operator [FIL_22] (rows=631 width=194) + Filter Operator [FIL_23] (rows=631 width=194) predicate:((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) - Merge Join Operator [MERGEJOIN_36] (rows=631 width=194) - Conds:RS_19._col0=RS_20._col0(Left Outer),Output:["_col0","_col1","_col2","_col3","_col5"] - <-Reducer 2 [SIMPLE_EDGE] llap - SHUFFLE [RS_19] - PartitionCols:_col0 - Merge Join Operator [MERGEJOIN_35] (rows=500 width=194) - Conds:(Inner),Output:["_col0","_col1","_col2","_col3"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] llap - PARTITION_ONLY_SHUFFLE [RS_16] - Select Operator [SEL_1] (rows=500 width=178) - Output:["_col0","_col1"] - TableScan [TS_0] (rows=500 width=178) - default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] - <-Reducer 6 [CUSTOM_SIMPLE_EDGE] llap - PARTITION_ONLY_SHUFFLE [RS_17] - Group By Operator [GBY_7] (rows=1 width=16) - Output:["_col0","_col1"],aggregations:["count(VALUE._col0)","count(VALUE._col1)"] - <-Map 5 [CUSTOM_SIMPLE_EDGE] llap - SHUFFLE [RS_6] - Group By Operator [GBY_5] (rows=1 width=16) - Output:["_col0","_col1"],aggregations:["count()","count(key)"] - Filter Operator [FIL_28] (rows=166 width=87) - predicate:(key > '2') - TableScan [TS_2] (rows=500 width=87) - default@src_cbo,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] - <-Reducer 7 [ONE_TO_ONE_EDGE] llap - FORWARD [RS_20] - PartitionCols:_col0 - Select Operator [SEL_15] (rows=83 width=91) - Output:["_col0","_col1"] - Group By Operator [GBY_14] (rows=83 width=87) - Output:["_col0"],keys:KEY._col0 - <-Map 5 [SIMPLE_EDGE] llap - SHUFFLE [RS_13] + Select Operator [SEL_22] (rows=631 width=194) + Output:["_col0","_col1","_col2","_col3","_col5"] + Merge Join Operator [MERGEJOIN_37] (rows=631 width=194) + Conds:(Inner),Output:["_col0","_col1","_col3","_col4","_col5"] + <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap + PARTITION_ONLY_SHUFFLE [RS_19] + Merge Join Operator [MERGEJOIN_36] (rows=631 width=178) + Conds:RS_16._col0=RS_17._col0(Left Outer),Output:["_col0","_col1","_col3"] + <-Map 1 [SIMPLE_EDGE] llap + SHUFFLE [RS_16] PartitionCols:_col0 - Group By Operator [GBY_12] (rows=83 width=87) - Output:["_col0"],keys:key - Filter Operator [FIL_29] (rows=166 width=87) + Select Operator [SEL_1] (rows=500 width=178) + Output:["_col0","_col1"] + TableScan [TS_0] (rows=500 width=178) + default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] + <-Reducer 6 [ONE_TO_ONE_EDGE] llap + FORWARD [RS_17] + PartitionCols:_col0 + Select Operator [SEL_8] (rows=83 width=91) + Output:["_col0","_col1"] + Group By Operator [GBY_7] (rows=83 width=87) + Output:["_col0"],keys:KEY._col0 + <-Map 5 [SIMPLE_EDGE] llap + PARTITION_ONLY_SHUFFLE [RS_6] + PartitionCols:_col0 + Group By Operator [GBY_5] (rows=83 width=87) + Output:["_col0"],keys:key + Filter Operator [FIL_29] (rows=166 width=87) + predicate:(key > '2') + TableScan [TS_2] (rows=500 width=87) + default@src_cbo,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] + <-Reducer 7 [CUSTOM_SIMPLE_EDGE] llap + PARTITION_ONLY_SHUFFLE [RS_20] + Group By Operator [GBY_14] (rows=1 width=16) + Output:["_col0","_col1"],aggregations:["count(VALUE._col0)","count(VALUE._col1)"] + <-Map 5 [CUSTOM_SIMPLE_EDGE] llap + PARTITION_ONLY_SHUFFLE [RS_13] + Group By Operator [GBY_12] (rows=1 width=16) + Output:["_col0","_col1"],aggregations:["count()","count(key)"] + Filter Operator [FIL_30] (rows=166 width=87) predicate:(key > '2') Please refer to the previous TableScan [TS_2] @@ -2697,8 +2707,8 @@ POSTHOOK: Input: default@part Plan optimized by CBO. Vertex dependency in root stage -Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE) -Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) +Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) +Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE) Reducer 4 <- Reducer 3 (SIMPLE_EDGE) Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE) @@ -2707,50 +2717,52 @@ Stage-0 limit:-1 Stage-1 Reducer 4 llap - File Output Operator [FS_31] - Select Operator [SEL_30] (rows=27 width=125) + File Output Operator [FS_32] + Select Operator [SEL_31] (rows=27 width=125) Output:["_col0","_col1"] <-Reducer 3 [SIMPLE_EDGE] llap - SHUFFLE [RS_29] - Select Operator [SEL_28] (rows=27 width=125) + SHUFFLE [RS_30] + Select Operator [SEL_29] (rows=27 width=125) Output:["_col0","_col1"] - Filter Operator [FIL_27] (rows=27 width=141) + Filter Operator [FIL_28] (rows=27 width=141) predicate:((_col2 = 0L) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) - Merge Join Operator [MERGEJOIN_37] (rows=27 width=141) - Conds:RS_24.UDFToDouble(_col1)=RS_25._col0(Left Outer),Output:["_col0","_col1","_col2","_col3","_col5"] - <-Reducer 6 [SIMPLE_EDGE] llap - SHUFFLE [RS_25] - PartitionCols:_col0 - Select Operator [SEL_20] (rows=1 width=12) - Output:["_col0","_col1"] - Group By Operator [GBY_7] (rows=1 width=16) - Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"] - <-Map 5 [CUSTOM_SIMPLE_EDGE] llap - PARTITION_ONLY_SHUFFLE [RS_6] - Group By Operator [GBY_5] (rows=1 width=16) - Output:["_col0","_col1"],aggregations:["sum(p_size)","count(p_size)"] - Filter Operator [FIL_33] (rows=8 width=4) - predicate:(p_size < 10) - TableScan [TS_2] (rows=26 width=4) - default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_size"] - <-Reducer 2 [SIMPLE_EDGE] llap - SHUFFLE [RS_24] - PartitionCols:UDFToDouble(_col1) - Merge Join Operator [MERGEJOIN_36] (rows=26 width=141) - Conds:(Inner),Output:["_col0","_col1","_col2","_col3"] - <-Reducer 6 [CUSTOM_SIMPLE_EDGE] llap - SHUFFLE [RS_22] - Group By Operator [GBY_12] (rows=1 width=16) - Output:["_col0","_col1"],aggregations:["count()","count(_col0)"] - Select Operator [SEL_8] (rows=1 width=16) - Output:["_col0"] + Select Operator [SEL_27] (rows=27 width=141) + Output:["_col0","_col1","_col2","_col3","_col5"] + Merge Join Operator [MERGEJOIN_38] (rows=27 width=141) + Conds:(Inner),Output:["_col0","_col1","_col3","_col4","_col5"] + <-Reducer 6 [CUSTOM_SIMPLE_EDGE] llap + PARTITION_ONLY_SHUFFLE [RS_25] + Group By Operator [GBY_19] (rows=1 width=16) + Output:["_col0","_col1"],aggregations:["count()","count(_col0)"] + Select Operator [SEL_15] (rows=1 width=16) + Output:["_col0"] + Group By Operator [GBY_7] (rows=1 width=16) + Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"] + <-Map 5 [CUSTOM_SIMPLE_EDGE] llap + PARTITION_ONLY_SHUFFLE [RS_6] + Group By Operator [GBY_5] (rows=1 width=16) + Output:["_col0","_col1"],aggregations:["sum(p_size)","count(p_size)"] + Filter Operator [FIL_35] (rows=8 width=4) + predicate:(p_size < 10) + TableScan [TS_2] (rows=26 width=4) + default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_size"] + <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap + PARTITION_ONLY_SHUFFLE [RS_24] + Merge Join Operator [MERGEJOIN_37] (rows=27 width=125) + Conds:RS_21.UDFToDouble(_col1)=RS_22._col0(Left Outer),Output:["_col0","_col1","_col3"] + <-Reducer 6 [SIMPLE_EDGE] llap + PARTITION_ONLY_SHUFFLE [RS_22] + PartitionCols:_col0 + Select Operator [SEL_8] (rows=1 width=12) + Output:["_col0","_col1"] Please refer to the previous Group By Operator [GBY_7] - <-Map 1 [CUSTOM_SIMPLE_EDGE] llap - PARTITION_ONLY_SHUFFLE [RS_21] - Select Operator [SEL_1] (rows=26 width=125) - Output:["_col0","_col1"] - TableScan [TS_0] (rows=26 width=125) - default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_size"] + <-Map 1 [SIMPLE_EDGE] llap + SHUFFLE [RS_21] + PartitionCols:UDFToDouble(_col1) + Select Operator [SEL_1] (rows=26 width=125) + Output:["_col0","_col1"] + TableScan [TS_0] (rows=26 width=125) + default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_size"] PREHOOK: query: explain select b.p_mfgr, min(p_retailprice) from part b diff --git a/ql/src/test/results/clientpositive/llap/lineage3.q.out b/ql/src/test/results/clientpositive/llap/lineage3.q.out index 9bec309c9c..67c2c592c8 100644 --- a/ql/src/test/results/clientpositive/llap/lineage3.q.out +++ b/ql/src/test/results/clientpositive/llap/lineage3.q.out @@ -180,7 +180,7 @@ PREHOOK: Input: default@src1 #### A masked pattern was here #### {"version":"1.0","engine":"tez","database":"default","hash":"94e9cc0a67801fe1503a3cb0c5029d59","queryText":"select * from src1 a\nwhere exists\n (select cint from alltypesorc b\n where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > 300.0D)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(a.key = a.key)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"b.ctinyint is not null","edgeType":"PREDICATE"},{"sources":[4,2],"targets":[0,1],"expression":"(UDFToDouble((UDFToInteger(b.ctinyint) + 300)) = UDFToDouble(a.key))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]} 311 val_311 -Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select key, value from src1 where key not in (select key+18 from src1) order by key PREHOOK: type: QUERY diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out index ac505a5c1e..e62eb0e574 100644 --- a/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out +++ b/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out @@ -446,7 +446,7 @@ POSTHOOK: Lineage: part_null_n1.p_partkey SCRIPT [] POSTHOOK: Lineage: part_null_n1.p_retailprice SCRIPT [] POSTHOOK: Lineage: part_null_n1.p_size SCRIPT [] POSTHOOK: Lineage: part_null_n1.p_type SCRIPT [] -Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select /*+ mapjoin(None)*/ * from part where p_name = (select p_name from part_null_n1 where p_name is null) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -466,30 +466,11 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Map 3 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) - Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Map 1 (XPROD_EDGE), Map 4 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: part - filterExpr: (p_name = null) (type: boolean) - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (p_name = null) (type: boolean) - Statistics: Num rows: 1 Data size: 619 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) - outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 3 Map Operator Tree: TableScan alias: part_null_n1 @@ -514,7 +495,42 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs + Map 4 + Map Operator Tree: + TableScan + alias: part + filterExpr: (p_name = null) (type: boolean) + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (p_name = null) (type: boolean) + Statistics: Num rows: 1 Data size: 619 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) + outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -525,10 +541,10 @@ STAGE PLANS: 0 1 2 - outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + outputColumnNames: _col2, _col4, _col5, _col6, _col7, _col8, _col9, _col10 Statistics: Num rows: 1 Data size: 959 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: int), null (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + expressions: _col2 (type: int), null (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 1 Data size: 959 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -538,22 +554,6 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 4 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: (sq_count_check(_col0) <= 1) (type: boolean) - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE - Select Operator - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 Fetch Operator @@ -561,7 +561,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Map Join MAPJOIN[24][bigTable=?] in task 'Map 1' is a cross product +Warning: Map Join MAPJOIN[24][bigTable=?] in task 'Map 3' is a cross product PREHOOK: query: explain select * from part where p_name = (select p_name from part_null_n1 where p_name is null) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -581,11 +581,36 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Map 1 <- Map 2 (BROADCAST_EDGE), Reducer 3 (BROADCAST_EDGE) - Reducer 3 <- Map 2 (CUSTOM_SIMPLE_EDGE) + Map 3 <- Map 1 (BROADCAST_EDGE), Reducer 2 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: part_null_n1 + filterExpr: p_name is null (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is null (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 3 Map Operator Tree: TableScan alias: part @@ -606,13 +631,13 @@ STAGE PLANS: 0 1 2 - outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + outputColumnNames: _col2, _col4, _col5, _col6, _col7, _col8, _col9, _col10 input vertices: - 1 Reducer 3 - 2 Map 2 + 0 Reducer 2 + 1 Map 1 Statistics: Num rows: 1 Data size: 959 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: int), null (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + expressions: _col2 (type: int), null (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 1 Data size: 959 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -624,32 +649,7 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Execution mode: vectorized, llap LLAP IO: no inputs - Map 2 - Map Operator Tree: - TableScan - alias: part_null_n1 - filterExpr: p_name is null (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: p_name is null (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Select Operator - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Execution mode: vectorized, llap - LLAP IO: no inputs - Reducer 3 + Reducer 2 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator diff --git a/ql/src/test/results/clientpositive/llap/optimize_join_ptp.q.out b/ql/src/test/results/clientpositive/llap/optimize_join_ptp.q.out index a68275b9e9..af55d4d5f3 100644 --- a/ql/src/test/results/clientpositive/llap/optimize_join_ptp.q.out +++ b/ql/src/test/results/clientpositive/llap/optimize_join_ptp.q.out @@ -55,31 +55,11 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: t1_n97 - filterExpr: (k < 15) (type: boolean) - Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (k < 15) (type: boolean) - Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: v (type: string), k (type: int) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: UDFToDouble(_col1) (type: double) - sort order: + - Map-reduce partition columns: UDFToDouble(_col1) (type: double) - Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col1 (type: int) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 3 Map Operator Tree: TableScan alias: t2_n60 @@ -90,10 +70,10 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 99 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: k (type: double) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: k + Statistics: Num rows: 1 Data size: 99 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - keys: _col0 (type: double) + keys: k (type: double) mode: hash outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE @@ -104,24 +84,62 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs + Map 4 + Map Operator Tree: + TableScan + alias: t1_n97 + filterExpr: (k < 15) (type: boolean) + Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (k < 15) (type: boolean) + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: v (type: string), k (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToDouble(_col1) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col1) (type: double) + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: int) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: double) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Semi Join 0 to 1 + Inner Join 0 to 1 keys: - 0 UDFToDouble(_col1) (type: double) - 1 _col0 (type: double) - outputColumnNames: _col0, _col1 + 0 _col0 (type: double) + 1 UDFToDouble(_col1) (type: double) + outputColumnNames: _col1, _col2 Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Select Operator + expressions: _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + 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 diff --git a/ql/src/test/results/clientpositive/llap/subquery_in.q.out b/ql/src/test/results/clientpositive/llap/subquery_in.q.out index 12e8c54480..3222e2f616 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_in.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_in.q.out @@ -21,10 +21,32 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: s1 + filterExpr: (key > '9') (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 4 Map Operator Tree: TableScan alias: src @@ -45,49 +67,41 @@ STAGE PLANS: value expressions: _col1 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs - Map 3 - Map Operator Tree: - TableScan - alias: s1 - filterExpr: (key > '9') (type: boolean) - Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (key > '9') (type: boolean) - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: key (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized, llap - LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Semi Join 0 to 1 + Inner Join 0 to 1 keys: 0 _col0 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1 + outputColumnNames: _col1, _col2 Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE + 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 @@ -1081,33 +1095,13 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) - Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) - Reducer 4 <- Map 6 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 7 <- Map 6 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: lineitem - filterExpr: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: l_partkey (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 5 Map Operator Tree: TableScan alias: li @@ -1128,6 +1122,27 @@ STAGE PLANS: value expressions: _col0 (type: int), _col2 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs + Map 4 + Map Operator Tree: + TableScan + alias: lineitem + filterExpr: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs Map 6 Map Operator Tree: TableScan @@ -1139,10 +1154,10 @@ STAGE PLANS: Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: l_orderkey (type: int) - outputColumnNames: _col0 - Statistics: Num rows: 14 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: l_orderkey + Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - keys: _col0 (type: int) + keys: l_orderkey (type: int) mode: hash outputColumnNames: _col0 Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE @@ -1154,48 +1169,35 @@ STAGE PLANS: Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - keys: KEY._col0 (type: int) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col1 (type: int) - outputColumnNames: _col0, _col1, _col3 + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: int) + key expressions: _col0 (type: int) sort order: + - Map-reduce partition columns: _col1 (type: int) + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col3 (type: int) - Reducer 4 + value expressions: _col1 (type: int), _col2 (type: int) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Semi Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col1 (type: int) + 0 _col0 (type: int) 1 _col0 (type: int) - outputColumnNames: _col0, _col3 + outputColumnNames: _col1, _col2 Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col3 (type: int) + expressions: _col1 (type: int), _col2 (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1205,6 +1207,32 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 7 Data size: 28 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: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Stage: Stage-0 Fetch Operator @@ -6004,7 +6032,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### 15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu -Warning: Shuffle Join MERGEJOIN[71][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[71][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 5' is a cross product PREHOOK: query: explain select * from part where p_size IN (select pp.p_size from part p join part pp on pp.p_type = p.p_type where part.p_type <> p.p_name) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -6023,9 +6051,9 @@ STAGE PLANS: #### A masked pattern was here #### Edges: Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) - Reducer 4 <- Map 3 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 5 <- Map 8 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) - Reducer 7 <- Map 6 (SIMPLE_EDGE) + Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) + Reducer 5 <- Reducer 4 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) + Reducer 8 <- Map 7 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -6063,33 +6091,14 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col1 (type: string) + value expressions: _col0 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Map 6 - Map Operator Tree: - TableScan - alias: part - filterExpr: p_type is not null (type: boolean) - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: p_type is not null (type: boolean) - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: p_type (type: string) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 8 Map Operator Tree: TableScan alias: pp @@ -6110,6 +6119,27 @@ STAGE PLANS: value expressions: _col1 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs + Map 7 + Map Operator Tree: + TableScan + alias: part + filterExpr: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 Execution mode: llap Reduce Operator Tree: @@ -6135,17 +6165,14 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - residual filter predicates: {(_col2 <> _col0)} - Statistics: Num rows: 338 Data size: 111202 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col3 + Statistics: Num rows: 28 Data size: 3500 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: string) - sort order: + - Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 338 Data size: 111202 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: string) + sort order: + Statistics: Num rows: 28 Data size: 3500 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col3 (type: int) Reducer 5 Execution mode: llap Reduce Operator Tree: @@ -6153,25 +6180,26 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 _col1 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col2, _col4 - Statistics: Num rows: 366 Data size: 39528 Basic stats: COMPLETE Column stats: COMPLETE + 0 + 1 + outputColumnNames: _col0, _col3, _col4 + residual filter predicates: {(_col4 <> _col0)} + Statistics: Num rows: 364 Data size: 83356 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col2 (type: string), _col4 (type: int) + expressions: _col4 (type: string), _col3 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 366 Data size: 39528 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 364 Data size: 39312 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator keys: _col0 (type: string), _col1 (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 183 Data size: 19764 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 182 Data size: 19656 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: int) sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: int) - Statistics: Num rows: 183 Data size: 19764 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 7 + Statistics: Num rows: 182 Data size: 19656 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -6190,7 +6218,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[71][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[71][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 5' is a cross product PREHOOK: query: select * from part where p_size IN (select pp.p_size from part p join part pp on pp.p_type = p.p_type where part.p_type <> p.p_name) PREHOOK: type: QUERY PREHOOK: Input: default@part diff --git a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out index 4f41f246f1..3839696882 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out @@ -557,34 +557,14 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE) - Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) + Reducer 6 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: b - filterExpr: (key > '8') (type: boolean) - Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (key > '8') (type: boolean) - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 5 Map Operator Tree: TableScan alias: src @@ -593,20 +573,16 @@ STAGE PLANS: Filter Operator predicate: (key > '8') (type: boolean) Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: key (type: string) + Group By Operator + keys: key (type: string) + mode: hash outputColumnNames: _col0 - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (key > '9') (type: boolean) Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE @@ -624,20 +600,54 @@ STAGE PLANS: value expressions: _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs + Map 7 + Map Operator Tree: + TableScan + alias: b + filterExpr: (key > '8') (type: boolean) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '8') (type: boolean) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Semi Join 0 to 1 + Inner Join 0 to 1 keys: 0 _col0 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1 + outputColumnNames: _col1, _col2 Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() - keys: _col0 (type: string), _col1 (type: string) + keys: _col1 (type: string), _col2 (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE @@ -647,7 +657,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: bigint) - Reducer 3 + Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -665,7 +675,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: bigint) Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string) - Reducer 4 + Reducer 5 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -777,12 +787,49 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Map 1 <- Map 3 (BROADCAST_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (BROADCAST_EDGE) - Reducer 4 <- Map 3 (SIMPLE_EDGE) + Map 4 <- Reducer 2 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 1 (SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE), Reducer 3 (BROADCAST_EDGE) #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: src + filterExpr: ((key > '8') or (key > '9')) (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '8') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 4 Map Operator Tree: TableScan alias: b @@ -797,17 +844,17 @@ STAGE PLANS: Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE Map Join Operator condition map: - Left Semi Join 0 to 1 + Inner Join 0 to 1 keys: 0 _col0 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1 + outputColumnNames: _col1, _col2 input vertices: - 1 Map 3 + 0 Reducer 2 Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() - keys: _col0 (type: string), _col1 (type: string) + keys: _col1 (type: string), _col2 (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE @@ -819,47 +866,50 @@ STAGE PLANS: value expressions: _col2 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs - Map 3 - Map Operator Tree: - TableScan - alias: src - filterExpr: ((key > '8') or (key > '9')) (type: boolean) - Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 83 Data size: 664 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (key > '8') (type: boolean) - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 83 Data size: 664 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: key (type: string) + expressions: _col1 (type: bigint) outputColumnNames: _col0 - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 83 Data size: 664 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - keys: _col0 (type: string) + keys: _col0 (type: bigint) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 41 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: string) + key expressions: _col0 (type: bigint) sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (key > '9') (type: boolean) - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - keys: key (type: string) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint) - Execution mode: vectorized, llap - LLAP IO: no inputs - Reducer 2 + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 41 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -879,7 +929,7 @@ STAGE PLANS: 1 _col0 (type: bigint) outputColumnNames: _col0, _col1, _col2 input vertices: - 1 Reducer 4 + 1 Reducer 3 Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false @@ -888,36 +938,6 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 4 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col1 (type: bigint) - outputColumnNames: _col1 - Statistics: Num rows: 83 Data size: 664 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: _col1 is not null (type: boolean) - Statistics: Num rows: 83 Data size: 664 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col1 (type: bigint) - outputColumnNames: _col0 - Statistics: Num rows: 83 Data size: 664 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: _col0 (type: bigint) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 41 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: bigint) - sort order: + - Map-reduce partition columns: _col0 (type: bigint) - Statistics: Num rows: 41 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out index d8d42ee10b..7b00d69754 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out @@ -258,7 +258,7 @@ POSTHOOK: Input: default@part_null 85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull 86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully 90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl -Warning: Shuffle Join MERGEJOIN[56][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND p_brand NOT IN (select p_name from part_null) PREHOOK: type: QUERY PREHOOK: Input: default@part_null @@ -276,12 +276,12 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE) - Reducer 7 <- Map 5 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Map 5 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (CUSTOM_SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) + Reducer 8 <- Map 4 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -298,32 +298,28 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col1 (type: string) + key expressions: _col3 (type: string) sort order: + - Map-reduce partition columns: _col1 (type: string) + Map-reduce partition columns: _col3 (type: string) Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs - Map 5 + Map 4 Map Operator Tree: TableScan alias: part_null - filterExpr: p_name is not null (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: p_name is not null (type: boolean) + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: p_name (type: string) - mode: hash - outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 184 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: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: p_name (type: string) outputColumnNames: p_name @@ -337,16 +333,19 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: p_name (type: string) - mode: hash - outputColumnNames: _col0 + Filter Operator + predicate: p_name is not null (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 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: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -354,16 +353,18 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 _col1 (type: string) + 0 _col3 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - sort order: + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: @@ -371,42 +372,29 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11 - Statistics: Num rows: 1 Data size: 1433 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col3 (type: string) - sort order: + - Map-reduce partition columns: _col3 (type: string) - Statistics: Num rows: 1 Data size: 1433 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: bigint), _col11 (type: bigint) - Reducer 4 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Outer Join 0 to 1 - keys: - 0 _col3 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13 - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col10 = 0L) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 1576 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 - Reducer 6 + 0 _col1 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 + Statistics: Num rows: 1 Data size: 1355 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13 + Statistics: Num rows: 1 Data size: 1355 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col10 = 0L) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) + Statistics: Num rows: 1 Data size: 1355 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 1355 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 1355 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 + Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -414,12 +402,17 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reducer 7 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -431,6 +424,23 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 7 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 385 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 1 Data size: 385 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: @@ -439,16 +449,10 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), true (type: boolean) - outputColumnNames: _col0, _col1 + Reduce Output Operator + sort order: Statistics: Num rows: 1 Data size: 184 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: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: boolean) + value expressions: _col0 (type: string) Stage: Stage-0 Fetch Operator @@ -456,7 +460,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[56][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product PREHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND p_brand NOT IN (select p_name from part_null) PREHOOK: type: QUERY PREHOOK: Input: default@part_null @@ -465,7 +469,7 @@ POSTHOOK: query: select * from part_null where p_name IN (select p_name from par POSTHOOK: type: QUERY POSTHOOK: Input: default@part_null #### A masked pattern was here #### -Warning: Shuffle Join MERGEJOIN[56][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND p_brand NOT IN (select p_type from part_null) PREHOOK: type: QUERY PREHOOK: Input: default@part_null @@ -483,12 +487,12 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) - Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE) - Reducer 8 <- Map 7 (CUSTOM_SIMPLE_EDGE) - Reducer 9 <- Map 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (CUSTOM_SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 9 <- Map 8 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -505,39 +509,28 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col1 (type: string) + key expressions: _col3 (type: string) sort order: + - Map-reduce partition columns: _col1 (type: string) + Map-reduce partition columns: _col3 (type: string) Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs - Map 5 + Map 4 Map Operator Tree: TableScan alias: part_null - filterExpr: p_name is not null (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: p_name is not null (type: boolean) + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: p_name (type: string) - mode: hash - outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 184 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: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 7 - Map Operator Tree: - TableScan - alias: part_null - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: p_type (type: string) outputColumnNames: p_type @@ -551,16 +544,27 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: p_type (type: string) - mode: hash - outputColumnNames: _col0 + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 8 + Map Operator Tree: + TableScan + alias: part_null + filterExpr: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 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: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -568,16 +572,18 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 _col1 (type: string) + 0 _col3 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - sort order: + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: @@ -585,42 +591,29 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11 - Statistics: Num rows: 1 Data size: 1433 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col3 (type: string) - sort order: + - Map-reduce partition columns: _col3 (type: string) - Statistics: Num rows: 1 Data size: 1433 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: bigint), _col11 (type: bigint) - Reducer 4 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Outer Join 0 to 1 - keys: - 0 _col3 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13 - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col10 = 0L) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 1576 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 - Reducer 6 + 0 _col1 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 + Statistics: Num rows: 1 Data size: 1355 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13 + Statistics: Num rows: 1 Data size: 1355 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col10 = 0L) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) + Statistics: Num rows: 1 Data size: 1355 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 1355 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 1355 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 + Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -628,12 +621,17 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reducer 8 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -645,6 +643,23 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 7 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 385 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 1 Data size: 385 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 9 Execution mode: vectorized, llap Reduce Operator Tree: @@ -653,16 +668,10 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), true (type: boolean) - outputColumnNames: _col0, _col1 + Reduce Output Operator + sort order: Statistics: Num rows: 1 Data size: 184 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: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: boolean) + value expressions: _col0 (type: string) Stage: Stage-0 Fetch Operator @@ -670,7 +679,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[56][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product PREHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND p_brand NOT IN (select p_type from part_null) PREHOOK: type: QUERY PREHOOK: Input: default@part_null @@ -705,7 +714,7 @@ POSTHOOK: Input: default@part_null 85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull 86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully 90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl -Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[55][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product PREHOOK: query: explain select * from part_null where p_brand IN (select p_brand from part_null) AND p_brand NOT IN (select p_name from part_null) PREHOOK: type: QUERY PREHOOK: Input: default@part_null @@ -723,12 +732,11 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) - Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE) - Reducer 8 <- Map 7 (CUSTOM_SIMPLE_EDGE) - Reducer 9 <- Map 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE), Reducer 6 (SIMPLE_EDGE) + Reducer 4 <- Map 3 (SIMPLE_EDGE) + Reducer 5 <- Map 3 (CUSTOM_SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) + Reducer 8 <- Map 7 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -752,17 +760,17 @@ STAGE PLANS: value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs - Map 5 + Map 3 Map Operator Tree: TableScan alias: part_null - filterExpr: p_brand is not null (type: boolean) + filterExpr: p_name is not null (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: p_brand is not null (type: boolean) + predicate: p_name is not null (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: p_brand (type: string) + keys: p_name (type: string) mode: hash outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE @@ -771,13 +779,6 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 7 - Map Operator Tree: - TableScan - alias: part_null - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: p_name (type: string) outputColumnNames: p_name @@ -791,11 +792,19 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 7 + Map Operator Tree: + TableScan + alias: part_null + filterExpr: p_brand is not null (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: p_name is not null (type: boolean) + predicate: p_brand is not null (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: p_name (type: string) + keys: p_brand (type: string) mode: hash outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE @@ -807,63 +816,37 @@ STAGE PLANS: Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col3 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - Reducer 3 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11 - Statistics: Num rows: 1 Data size: 1433 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col3 (type: string) - sort order: + - Map-reduce partition columns: _col3 (type: string) - Statistics: Num rows: 1 Data size: 1433 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: bigint), _col11 (type: bigint) - Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: Left Outer Join 0 to 1 + Inner Join 0 to 2 keys: 0 _col3 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13 - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col10 = 0L) or (_col13 is null and (_col11 >= _col10))) (type: boolean) - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 1576 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 1576 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 - Reducer 6 + 2 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 + Statistics: Num rows: 2 Data size: 2464 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13 + Statistics: Num rows: 2 Data size: 2464 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col10 = 0L) or (_col13 is null and (_col11 >= _col10))) (type: boolean) + Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 1232 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 + Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -871,12 +854,17 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reducer 8 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -888,7 +876,24 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 9 + Reducer 6 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 385 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 1 Data size: 385 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -896,16 +901,10 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), true (type: boolean) - outputColumnNames: _col0, _col1 + Reduce Output Operator + sort order: Statistics: Num rows: 1 Data size: 184 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: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: boolean) + value expressions: _col0 (type: string) Stage: Stage-0 Fetch Operator @@ -913,7 +912,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[55][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product PREHOOK: query: select * from part_null where p_brand IN (select p_brand from part_null) AND p_brand NOT IN (select p_name from part_null) PREHOOK: type: QUERY PREHOOK: Input: default@part_null @@ -922,7 +921,7 @@ POSTHOOK: query: select * from part_null where p_brand IN (select p_brand from p POSTHOOK: type: QUERY POSTHOOK: Input: default@part_null #### A masked pattern was here #### -Warning: Shuffle Join MERGEJOIN[56][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part_null where p_name NOT IN (select c from tempty) AND p_brand IN (select p_brand from part_null) PREHOOK: type: QUERY PREHOOK: Input: default@part_null @@ -942,11 +941,11 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) Reducer 4 <- Map 8 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) - Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE) - Reducer 7 <- Map 5 (SIMPLE_EDGE) + Reducer 6 <- Map 5 (SIMPLE_EDGE) + Reducer 7 <- Map 5 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -963,9 +962,11 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - sort order: + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Map 5 @@ -973,6 +974,16 @@ STAGE PLANS: TableScan alias: tempty Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: c (type: char(2)) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: char(2)) + sort order: + + Map-reduce partition columns: _col0 (type: char(2)) + Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: c (type: char(2)) outputColumnNames: c @@ -986,16 +997,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: c (type: char(2)) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: char(2)) - sort order: + - Map-reduce partition columns: _col0 (type: char(2)) - Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs Map 8 @@ -1028,42 +1029,44 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 1 Data size: 1223 Basic stats: COMPLETE Column stats: NONE + 0 _col1 (type: string) + 1 CAST( _col0 AS STRING) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col1 (type: string) - sort order: + - Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 1 Data size: 1223 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: bigint) + sort order: + Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col1 (type: string) - 1 CAST( _col0 AS STRING) (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 - Statistics: Num rows: 1 Data size: 1345 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col12 is null and _col1 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) - Statistics: Num rows: 1 Data size: 1345 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 1345 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col3 (type: string) - sort order: + - Map-reduce partition columns: _col3 (type: string) - Statistics: Num rows: 1 Data size: 1345 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 + Statistics: Num rows: 1 Data size: 1335 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + Statistics: Num rows: 1 Data size: 1335 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col12 is null and _col1 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Statistics: Num rows: 1 Data size: 1335 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 1335 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 1 Data size: 1335 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Reducer 4 Execution mode: llap Reduce Operator Tree: @@ -1074,27 +1077,15 @@ STAGE PLANS: 0 _col3 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 1468 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 1468 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 Reducer 6 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -1112,6 +1103,18 @@ STAGE PLANS: Map-reduce partition columns: CAST( _col0 AS STRING) (type: string) Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -1119,7 +1122,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[56][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part_null where p_name NOT IN (select c from tempty) AND p_brand IN (select p_brand from part_null) PREHOOK: type: QUERY PREHOOK: Input: default@part_null @@ -2950,7 +2953,7 @@ POSTHOOK: Input: default@part_null 85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull 86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully 90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl -Warning: Shuffle Join MERGEJOIN[82][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product +Warning: Shuffle Join MERGEJOIN[83][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product PREHOOK: query: explain select p.p_partkey, li.l_suppkey from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey where li.l_linenumber = 1 and @@ -2974,36 +2977,14 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Map 9 (CUSTOM_SIMPLE_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE) - Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) - Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) - Reducer 7 <- Map 6 (XPROD_EDGE), Reducer 10 (XPROD_EDGE) - Reducer 8 <- Reducer 10 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 6 <- Map 5 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 9 <- Map 8 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: lineitem - filterExpr: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: l_partkey (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 5 Map Operator Tree: TableScan alias: li @@ -3024,7 +3005,32 @@ STAGE PLANS: value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs - Map 6 + Map 4 + Map Operator Tree: + TableScan + alias: lineitem + filterExpr: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: l_partkey (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 5 Map Operator Tree: TableScan alias: lineitem @@ -3038,12 +3044,14 @@ STAGE PLANS: outputColumnNames: _col0, _col2 Statistics: Num rows: 2 Data size: 206 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col2 (type: double) + sort order: + + Map-reduce partition columns: _col2 (type: double) Statistics: Num rows: 2 Data size: 206 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col2 (type: double) + value expressions: _col0 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs - Map 9 + Map 8 Map Operator Tree: TableScan alias: lineitem @@ -3063,68 +3071,28 @@ STAGE PLANS: value expressions: _col0 (type: double), _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs - Reducer 10 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: sum(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: (_col0 / _col1) (type: double) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col0) - mode: complete - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Select Operator - expressions: (_col0 / _col1) (type: double), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double) - sort order: + - Map-reduce partition columns: _col0 (type: double) - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) Reducer 2 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - keys: KEY._col0 (type: int) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Semi Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col1 (type: int) - outputColumnNames: _col0, _col1, _col3, _col4 + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col1 (type: int), _col4 (type: int) - sort order: ++ - Map-reduce partition columns: _col1 (type: int), _col4 (type: int) + Select Operator + expressions: _col1 (type: int), _col0 (type: int), _col2 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4 Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col3 (type: int) - Reducer 4 + Reduce Output Operator + key expressions: _col1 (type: int), _col4 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: int), _col4 (type: int) + Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col3 (type: int) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -3146,51 +3114,84 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 7 + Reducer 6 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col2, _col4, _col5 - Statistics: Num rows: 2 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col2 (type: double) + 1 _col0 (type: double) + outputColumnNames: _col0, _col2, _col5 + Statistics: Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col2 (type: double) - sort order: + - Map-reduce partition columns: _col2 (type: double) - Statistics: Num rows: 2 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col4 (type: bigint), _col5 (type: bigint) - Reducer 8 + sort order: + Statistics: Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: double), _col5 (type: boolean) + Reducer 7 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col2 (type: double) - 1 _col0 (type: double) - outputColumnNames: _col0, _col2, _col4, _col5, _col7 + 0 + 1 + outputColumnNames: _col0, _col2, _col5, _col6, _col7 Statistics: Num rows: 2 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col4 = 0L) or (_col7 is null and _col2 is not null and (_col5 >= _col4))) (type: boolean) + Select Operator + expressions: _col0 (type: int), _col2 (type: double), _col6 (type: bigint), _col7 (type: bigint), _col5 (type: boolean) + outputColumnNames: _col0, _col2, _col4, _col5, _col7 Statistics: Num rows: 2 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), 1 (type: int) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: _col0 (type: int), _col1 (type: int) - mode: hash + Filter Operator + predicate: ((_col4 = 0L) or (_col7 is null and _col2 is not null and (_col5 >= _col4))) (type: boolean) + Statistics: Num rows: 2 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), 1 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - 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: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int), _col1 (type: int) + mode: hash + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 9 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: (_col0 / _col1) (type: double), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Select Operator + expressions: (_col0 / _col1) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -3198,7 +3199,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[82][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product +Warning: Shuffle Join MERGEJOIN[83][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product PREHOOK: query: select p.p_partkey, li.l_suppkey from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey where li.l_linenumber = 1 and @@ -3953,7 +3954,7 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tnull POSTHOOK: Lineage: tnull.c SCRIPT [] POSTHOOK: Lineage: tnull.i SCRIPT [] -Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where p_partkey = 3 OR p_size NOT IN (select i from tnull) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -3973,10 +3974,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -3989,9 +3990,11 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col5 (type: int) + sort order: + + Map-reduce partition columns: _col5 (type: int) Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Map 4 @@ -3999,6 +4002,16 @@ STAGE PLANS: TableScan alias: tnull Statistics: Num rows: 3 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: i (type: int) outputColumnNames: i @@ -4012,16 +4025,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: i (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -4029,56 +4032,46 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col5 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 27 Data size: 16721 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col5 (type: int) - sort order: + - Map-reduce partition columns: _col5 (type: int) - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: bigint) + sort order: + Statistics: Num rows: 27 Data size: 16721 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col5 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 Statistics: Num rows: 27 Data size: 17153 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col0 = 3) or CASE WHEN ((_col9 = 0L)) THEN (true) WHEN (_col12 is not null) THEN (false) WHEN (_col5 is null) THEN (null) WHEN ((_col10 < _col9)) THEN (null) ELSE (true) END) (type: boolean) - Statistics: Num rows: 14 Data size: 8898 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 14 Data size: 8666 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + Statistics: Num rows: 27 Data size: 17153 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col0 = 3) or CASE WHEN ((_col9 = 0L)) THEN (true) WHEN (_col12 is not null) THEN (false) WHEN (_col5 is null) THEN (null) WHEN ((_col10 < _col9)) THEN (null) ELSE (true) END) (type: boolean) + Statistics: Num rows: 14 Data size: 8898 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 14 Data size: 8666 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 14 Data size: 8666 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -4096,6 +4089,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -4103,7 +4108,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part where p_partkey = 3 OR p_size NOT IN (select i from tnull) PREHOOK: type: QUERY PREHOOK: Input: default@part diff --git a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out index d981733bd3..4c63aa5aab 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out @@ -1,4 +1,4 @@ -Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from src @@ -28,10 +28,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -44,9 +44,11 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col1 (type: string) + value expressions: _col1 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Map 4 @@ -55,18 +57,6 @@ STAGE PLANS: alias: s1 filterExpr: ((key > '2') or (key > '2')) (type: boolean) Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (key > '2') (type: boolean) - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(key) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) Filter Operator predicate: (key > '2') (type: boolean) Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE @@ -80,6 +70,18 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '2') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(key) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -87,56 +89,46 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 631 Data size: 112846 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string), _col2 (type: bigint), _col3 (type: bigint) + sort order: + Statistics: Num rows: 631 Data size: 112846 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + 0 + 1 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 Statistics: Num rows: 631 Data size: 122942 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 Statistics: Num rows: 631 Data size: 122942 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 631 Data size: 112318 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 631 Data size: 122942 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 631 Data size: 112318 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 631 Data size: 112318 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -154,6 +146,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 83 Data size: 7553 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -161,7 +165,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from src where src.key not in ( select key from src s1 where s1.key > '2') @@ -583,7 +587,7 @@ Manufacturer#4 almond azure aquamarine papaya violet 12 Manufacturer#5 almond antique blue firebrick mint 31 Manufacturer#5 almond aquamarine dodger light gainsboro 46 Manufacturer#5 almond azure blanched chiffon midnight 23 -Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[46][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select p_name, p_size from @@ -615,8 +619,8 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) Reducer 5 <- Map 4 (SIMPLE_EDGE) Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) Reducer 7 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) @@ -632,7 +636,9 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: UDFToDouble(_col1) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col1) (type: double) Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: int) Execution mode: vectorized, llap @@ -655,43 +661,45 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 26 Data size: 3666 Basic stats: COMPLETE Column stats: COMPLETE + 0 UDFToDouble(_col1) (type: double) + 1 _col0 (type: double) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 27 Data size: 3383 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: UDFToDouble(_col1) (type: double) - sort order: + - Map-reduce partition columns: UDFToDouble(_col1) (type: double) - Statistics: Num rows: 26 Data size: 3666 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint) + sort order: + Statistics: Num rows: 27 Data size: 3383 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: int), _col3 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 UDFToDouble(_col1) (type: double) - 1 _col0 (type: double) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + 0 + 1 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 Statistics: Num rows: 27 Data size: 3815 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col2 = 0L) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) + Select Operator + expressions: _col0 (type: string), _col1 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 Statistics: Num rows: 27 Data size: 3815 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), _col1 (type: int) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 27 Data size: 3375 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col2 = 0L) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 27 Data size: 3815 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 27 Data size: 3375 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 27 Data size: 3375 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: @@ -774,6 +782,24 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: (UDFToDouble(_col0) / _col1) (type: double), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -794,24 +820,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 7 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: sum(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: (UDFToDouble(_col0) / _col1) (type: double), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double) - sort order: + - Map-reduce partition columns: _col0 (type: double) - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) Stage: Stage-0 Fetch Operator @@ -819,7 +827,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[46][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[48][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select p_name, p_size from part where part.p_size not in @@ -1237,7 +1245,7 @@ Manufacturer#5 almond antique medium spring khaki 6 Manufacturer#5 almond azure blanched chiffon midnight 23 Manufacturer#5 almond antique blue firebrick mint 31 Manufacturer#5 almond aquamarine dodger light gainsboro 46 -Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product PREHOOK: query: select li.l_partkey, count(*) from lineitem li where li.l_linenumber = 1 and @@ -1270,7 +1278,7 @@ POSTHOOK: Input: default@lineitem 88035 1 123076 1 182052 1 -Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from src where not src.key in ( select key from src s1 where s1.key > '2') @@ -1432,7 +1440,7 @@ POSTHOOK: Input: default@t1_v POSTHOOK: Output: database:default POSTHOOK: Output: default@T2_v POSTHOOK: Lineage: T2_v.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] -Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from T1_v where T1_v.key not in (select T2_v.key from T2_v) @@ -1458,10 +1466,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) - Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 5 <- Map 1 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) + Reducer 4 <- Map 1 (SIMPLE_EDGE) + Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -1480,22 +1488,10 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string) - Select Operator - expressions: CASE WHEN ((key > '104')) THEN (null) ELSE (key) END (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col0) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) Filter Operator predicate: ((key < '11') and CASE WHEN ((key > '104')) THEN (null) ELSE ((key < '11')) END) (type: boolean) Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE @@ -1513,6 +1509,22 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 41 Data size: 7544 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < '11') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: CASE WHEN ((key > '104')) THEN (null) ELSE (key) END (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -1520,56 +1532,46 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 166 Data size: 17098 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 230 Data size: 20270 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 166 Data size: 17098 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint), _col2 (type: bigint) + sort order: + Statistics: Num rows: 230 Data size: 20270 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col2 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col4 + 0 + 1 + outputColumnNames: _col0, _col2, _col3, _col4 Statistics: Num rows: 230 Data size: 23950 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Select Operator + expressions: _col0 (type: string), _col3 (type: bigint), _col4 (type: bigint), _col2 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col4 Statistics: Num rows: 230 Data size: 23950 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 230 Data size: 20010 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Statistics: Num rows: 230 Data size: 23950 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 230 Data size: 20010 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 230 Data size: 20010 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 4 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -1587,6 +1589,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 41 Data size: 7708 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -1594,7 +1608,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from T1_v where T1_v.key not in (select T2_v.key from T2_v) PREHOOK: type: QUERY @@ -1810,7 +1824,7 @@ POSTHOOK: Input: default@part 42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl 195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de 144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about -Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_partkey PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -1828,12 +1842,12 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) Reducer 4 <- Reducer 3 (SIMPLE_EDGE) Reducer 6 <- Map 5 (SIMPLE_EDGE) - Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Reducer 6 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE) + Reducer 8 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -1846,7 +1860,9 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: (_col5 - 1) (type: int) + sort order: + + Map-reduce partition columns: (_col5 - 1) (type: int) Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap @@ -1879,41 +1895,43 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE + 0 (_col5 - 1) (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 33 Data size: 20459 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: (_col5 - 1) (type: int) - sort order: + - Map-reduce partition columns: (_col5 - 1) (type: int) - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: bigint) + sort order: + Statistics: Num rows: 33 Data size: 20459 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 (_col5 - 1) (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int) - sort order: + + Filter Operator + predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: @@ -1940,16 +1958,7 @@ STAGE PLANS: Select Operator expressions: _col1 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col1) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator keys: _col1 (type: int) mode: hash @@ -1960,19 +1969,16 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 7 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -1990,6 +1996,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 8 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -1997,7 +2015,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_partkey PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2022,7 +2040,7 @@ POSTHOOK: Input: default@part 132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even 144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about 192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir -Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where (p_partkey*p_size) NOT IN (select min(p_partkey) from part group by p_type) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2040,11 +2058,11 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) Reducer 5 <- Map 4 (SIMPLE_EDGE) - Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) - Reducer 7 <- Reducer 5 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (SIMPLE_EDGE) + Reducer 7 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -2057,7 +2075,9 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: (_col0 * _col5) (type: int) + sort order: + + Map-reduce partition columns: (_col0 * _col5) (type: int) Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap @@ -2090,43 +2110,45 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE + 0 (_col0 * _col5) (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 32 Data size: 19836 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: (_col0 * _col5) (type: int) - sort order: + - Map-reduce partition columns: (_col0 * _col5) (type: int) - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: bigint) + sort order: + Statistics: Num rows: 32 Data size: 19836 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 (_col0 * _col5) (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 Statistics: Num rows: 32 Data size: 20348 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col12 is null and _col0 is not null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 Statistics: Num rows: 32 Data size: 20348 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 32 Data size: 19808 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col12 is null and _col0 is not null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Statistics: Num rows: 32 Data size: 20348 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 32 Data size: 19808 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 32 Data size: 19808 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: @@ -2139,16 +2161,7 @@ STAGE PLANS: Select Operator expressions: _col1 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col1) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator keys: _col1 (type: int) mode: hash @@ -2159,19 +2172,16 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 6 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -2189,6 +2199,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -2196,7 +2218,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part where (p_partkey*p_size) NOT IN (select min(p_partkey) from part group by p_type) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2206,30 +2228,30 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### 17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve -17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the -121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h -121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h -42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl -146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref -155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra -33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful 49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the 110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously -15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu -45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful -105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ -90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl -78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith -48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i 112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car -86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about 65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr -85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful 132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h 195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de -192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir -144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl 191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i PREHOOK: query: explain select count(*) as c from part as e where p_size + 100 NOT IN (select p_partkey from part where p_name = e.p_name) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2467,7 +2489,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### 26 -Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where floor(p_retailprice) NOT IN (select floor(min(p_retailprice)) from part group by p_type) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2485,11 +2507,11 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) Reducer 5 <- Map 4 (SIMPLE_EDGE) - Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) - Reducer 7 <- Reducer 5 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (SIMPLE_EDGE) + Reducer 7 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -2502,7 +2524,9 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: floor(_col7) (type: bigint) + sort order: + + Map-reduce partition columns: floor(_col7) (type: bigint) Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap @@ -2535,43 +2559,45 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE + 0 floor(_col7) (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 32 Data size: 19836 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: floor(_col7) (type: bigint) - sort order: + - Map-reduce partition columns: floor(_col7) (type: bigint) - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: bigint) + sort order: + Statistics: Num rows: 32 Data size: 19836 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 floor(_col7) (type: bigint) - 1 _col0 (type: bigint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 Statistics: Num rows: 32 Data size: 20348 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col12 is null and _col7 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 Statistics: Num rows: 32 Data size: 20348 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 32 Data size: 19808 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col12 is null and _col7 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Statistics: Num rows: 32 Data size: 20348 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 32 Data size: 19808 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 32 Data size: 19808 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: @@ -2584,16 +2610,7 @@ STAGE PLANS: Select Operator expressions: floor(_col1) (type: bigint) outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col0) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator keys: _col0 (type: bigint) mode: hash @@ -2604,19 +2621,16 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: bigint) Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 6 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -2634,14 +2648,26 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: bigint) Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - - Stage: Stage-0 + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + + Stage: Stage-0 Fetch Operator limit: -1 Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part where floor(p_retailprice) NOT IN (select floor(min(p_retailprice)) from part group by p_type) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -3571,7 +3597,7 @@ almond azure aquamarine papaya violet almond antique medium spring khaki almond aquamarine sandy cyan gainsboro almond antique olive coral navajo -Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select key, count(*) from src where value NOT IN (select key from src) group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key ) PREHOOK: type: QUERY PREHOOK: Input: default@src @@ -3589,12 +3615,12 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) Reducer 4 <- Reducer 3 (SIMPLE_EDGE) Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) - Reducer 7 <- Map 6 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Map 6 (SIMPLE_EDGE) + Reducer 7 <- Map 6 (SIMPLE_EDGE) + Reducer 8 <- Map 6 (CUSTOM_SIMPLE_EDGE) Reducer 9 <- Map 6 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: @@ -3608,9 +3634,11 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col1 (type: string) + value expressions: _col0 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Map 6 @@ -3618,6 +3646,16 @@ STAGE PLANS: TableScan alias: src Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: key (type: string) outputColumnNames: key @@ -3631,16 +3669,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: key (type: string) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (key = '90') (type: boolean) Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE @@ -3665,48 +3693,50 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 907 Data size: 163078 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: string) - sort order: + - Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col2 (type: bigint), _col3 (type: bigint) + sort order: + Statistics: Num rows: 907 Data size: 163078 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col1 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + 0 + 1 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 Statistics: Num rows: 907 Data size: 177590 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col2 = 0L) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 Statistics: Num rows: 907 Data size: 177590 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string) - outputColumnNames: _col0 + Filter Operator + predicate: ((_col2 = 0L) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) Statistics: Num rows: 907 Data size: 177590 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 907 Data size: 177590 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint) + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: @@ -3744,18 +3774,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Reducer 7 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3773,6 +3791,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 8 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 9 Execution mode: vectorized, llap Reduce Operator Tree: @@ -3810,7 +3840,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select key, count(*) from src where value NOT IN (select key from src) group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key ) PREHOOK: type: QUERY PREHOOK: Input: default@src @@ -4146,7 +4176,7 @@ POSTHOOK: Input: default@src 431 3 403 3 193 3 -Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -4164,12 +4194,12 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) Reducer 4 <- Reducer 3 (SIMPLE_EDGE) Reducer 6 <- Map 5 (SIMPLE_EDGE) - Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Reducer 6 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE) + Reducer 8 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -4182,7 +4212,9 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: (_col5 - 1) (type: int) + sort order: + + Map-reduce partition columns: (_col5 - 1) (type: int) Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap @@ -4215,41 +4247,43 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE + 0 (_col5 - 1) (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 33 Data size: 20459 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: (_col5 - 1) (type: int) - sort order: + - Map-reduce partition columns: (_col5 - 1) (type: int) - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: bigint) + sort order: + Statistics: Num rows: 33 Data size: 20459 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 (_col5 - 1) (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col3 (type: string) - sort order: + + Filter Operator + predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: @@ -4276,16 +4310,7 @@ STAGE PLANS: Select Operator expressions: _col1 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col1) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator keys: _col1 (type: int) mode: hash @@ -4296,19 +4321,16 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 7 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -4326,6 +4348,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 8 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -4333,7 +4367,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -4342,8 +4376,8 @@ POSTHOOK: query: select * from part where (p_size-1) NOT IN (select min(p_size) POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### -65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr 85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr 110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously 105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ 132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even @@ -4353,12 +4387,12 @@ POSTHOOK: Input: default@part 17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the 33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful 49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick -45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful 48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful 42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl -192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir 78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith -Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand, p_partkey limit 4 PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -4376,12 +4410,12 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) Reducer 4 <- Reducer 3 (SIMPLE_EDGE) Reducer 6 <- Map 5 (SIMPLE_EDGE) - Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Reducer 6 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE) + Reducer 8 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -4394,7 +4428,9 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: (_col5 - 1) (type: int) + sort order: + + Map-reduce partition columns: (_col5 - 1) (type: int) Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap @@ -4427,42 +4463,44 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE + 0 (_col5 - 1) (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 33 Data size: 20459 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: (_col5 - 1) (type: int) - sort order: + - Map-reduce partition columns: (_col5 - 1) (type: int) - Statistics: Num rows: 26 Data size: 16510 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: bigint) + sort order: + Statistics: Num rows: 33 Data size: 20459 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 (_col5 - 1) (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12 Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col10 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col3 (type: string), _col0 (type: int) - sort order: ++ + Filter Operator + predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Statistics: Num rows: 33 Data size: 20987 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE - TopN Hash Memory Usage: 0.1 - value expressions: _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Reduce Output Operator + key expressions: _col3 (type: string), _col0 (type: int) + sort order: ++ + Statistics: Num rows: 33 Data size: 20427 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: @@ -4492,16 +4530,7 @@ STAGE PLANS: Select Operator expressions: _col1 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col1) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator keys: _col1 (type: int) mode: hash @@ -4512,23 +4541,20 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 8 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - keys: KEY._col0 (type: int) + keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE @@ -4542,6 +4568,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 8 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -4549,7 +4587,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand, p_partkey limit 4 PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -4562,7 +4600,7 @@ POSTHOOK: Input: default@part 85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull 110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously 105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ -Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[45][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from src where key NOT IN (select p_name from part UNION ALL select p_brand from part) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -4586,10 +4624,10 @@ STAGE PLANS: Map 4 <- Union 5 (CONTAINS) Map 7 <- Union 5 (CONTAINS) Map 8 <- Union 9 (CONTAINS) - Reducer 10 <- Union 9 (SIMPLE_EDGE) - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) - Reducer 3 <- Reducer 10 (ONE_TO_ONE_EDGE), Reducer 2 (SIMPLE_EDGE) - Reducer 6 <- Union 5 (CUSTOM_SIMPLE_EDGE) + Reducer 10 <- Union 9 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 10 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) + Reducer 6 <- Union 5 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -4602,9 +4640,11 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col1 (type: string) + value expressions: _col1 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Map 11 @@ -4617,15 +4657,14 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 26 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - keys: _col0 (type: string) + aggregations: count(), count(_col0) mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs Map 4 @@ -4638,14 +4677,15 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - aggregations: count(), count(_col0) + keys: _col0 (type: string) mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Map 7 @@ -4658,14 +4698,15 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 26 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - aggregations: count(), count(_col0) + keys: _col0 (type: string) mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Map 8 @@ -4678,89 +4719,90 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - keys: _col0 (type: string) + aggregations: count(), count(_col0) mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs Reducer 10 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator - keys: KEY._col0 (type: string) + aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 25 Data size: 3125 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 25 Data size: 3125 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 2 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 539 Data size: 96102 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string), _col2 (type: bigint), _col3 (type: bigint) + sort order: + Statistics: Num rows: 539 Data size: 96102 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + 0 + 1 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 Statistics: Num rows: 539 Data size: 104726 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 Statistics: Num rows: 539 Data size: 104726 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 539 Data size: 95942 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 539 Data size: 104726 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 539 Data size: 95942 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 539 Data size: 95942 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 25 Data size: 3125 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 25 Data size: 3125 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) Union 5 Vertex: Union 5 Union 9 @@ -4772,7 +4814,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[45][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from src where key NOT IN (select p_name from part UNION ALL select p_brand from part) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -4783,506 +4825,506 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@part POSTHOOK: Input: default@src #### A masked pattern was here #### -0 val_0 -0 val_0 -0 val_0 -10 val_10 +92 val_92 +446 val_446 +393 val_393 +368 val_368 +33 val_33 +242 val_242 +242 val_242 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +9 val_9 +85 val_85 +480 val_480 +480 val_480 +480 val_480 +455 val_455 +396 val_396 +396 val_396 +396 val_396 +377 val_377 +226 val_226 +18 val_18 +18 val_18 100 val_100 100 val_100 -103 val_103 -103 val_103 -104 val_104 -104 val_104 -105 val_105 -11 val_11 -111 val_111 -113 val_113 -113 val_113 -114 val_114 -116 val_116 -118 val_118 -118 val_118 -119 val_119 -119 val_119 -119 val_119 -12 val_12 -12 val_12 -120 val_120 -120 val_120 -125 val_125 -125 val_125 -126 val_126 +54 val_54 +460 val_460 +435 val_435 +382 val_382 +382 val_382 +153 val_153 128 val_128 128 val_128 128 val_128 -129 val_129 -129 val_129 +96 val_96 +70 val_70 +70 val_70 +70 val_70 +454 val_454 +454 val_454 +454 val_454 +448 val_448 +429 val_429 +429 val_429 +395 val_395 +395 val_395 +322 val_322 +322 val_322 +298 val_298 +298 val_298 +298 val_298 +244 val_244 +191 val_191 +191 val_191 +172 val_172 +172 val_172 +166 val_166 +11 val_11 +492 val_492 +492 val_492 +467 val_467 +400 val_400 +389 val_389 +37 val_37 +37 val_37 +341 val_341 +316 val_316 +316 val_316 +316 val_316 +263 val_263 +238 val_238 +238 val_238 +219 val_219 +219 val_219 +87 val_87 +53 val_53 +413 val_413 +413 val_413 +360 val_360 +335 val_335 +282 val_282 +282 val_282 +257 val_257 +179 val_179 +179 val_179 131 val_131 -133 val_133 -134 val_134 -134 val_134 -136 val_136 -137 val_137 -137 val_137 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -143 val_143 -145 val_145 -146 val_146 -146 val_146 -149 val_149 -149 val_149 -15 val_15 -15 val_15 -150 val_150 -152 val_152 -152 val_152 -153 val_153 -155 val_155 -156 val_156 -157 val_157 -158 val_158 -160 val_160 -162 val_162 -163 val_163 -164 val_164 -164 val_164 -165 val_165 -165 val_165 -166 val_166 -167 val_167 -167 val_167 -167 val_167 -168 val_168 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -17 val_17 -170 val_170 -172 val_172 -172 val_172 -174 val_174 -174 val_174 -175 val_175 -175 val_175 -176 val_176 -176 val_176 -177 val_177 -178 val_178 -179 val_179 -179 val_179 -18 val_18 -18 val_18 -180 val_180 -181 val_181 -183 val_183 -186 val_186 -187 val_187 -187 val_187 -187 val_187 -189 val_189 -19 val_19 -190 val_190 -191 val_191 -191 val_191 -192 val_192 -193 val_193 -193 val_193 -193 val_193 -194 val_194 -195 val_195 -195 val_195 -196 val_196 -197 val_197 -197 val_197 -199 val_199 -199 val_199 -199 val_199 -2 val_2 -20 val_20 -200 val_200 -200 val_200 -201 val_201 -202 val_202 +95 val_95 +95 val_95 +432 val_432 +407 val_407 +373 val_373 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +28 val_28 +222 val_222 203 val_203 203 val_203 -205 val_205 -205 val_205 -207 val_207 -207 val_207 +150 val_150 +125 val_125 +125 val_125 +119 val_119 +119 val_119 +119 val_119 +66 val_66 +58 val_58 +58 val_58 +462 val_462 +462 val_462 +443 val_443 +437 val_437 +418 val_418 +384 val_384 +384 val_384 +384 val_384 +365 val_365 +311 val_311 +311 val_311 +311 val_311 +287 val_287 +233 val_233 +233 val_233 +214 val_214 208 val_208 208 val_208 208 val_208 -209 val_209 -209 val_209 -213 val_213 -213 val_213 -214 val_214 +180 val_180 +155 val_155 +136 val_136 +470 val_470 +44 val_44 +392 val_392 +367 val_367 +367 val_367 +289 val_289 +241 val_241 216 val_216 216 val_216 -217 val_217 -217 val_217 -218 val_218 -219 val_219 -219 val_219 -221 val_221 -221 val_221 -222 val_222 -223 val_223 -223 val_223 -224 val_224 -224 val_224 -226 val_226 -228 val_228 -229 val_229 -229 val_229 -230 val_230 -230 val_230 -230 val_230 -230 val_230 -230 val_230 -233 val_233 -233 val_233 -235 val_235 -237 val_237 -237 val_237 -238 val_238 -238 val_238 -239 val_239 -239 val_239 -24 val_24 -24 val_24 -241 val_241 -242 val_242 -242 val_242 -244 val_244 -247 val_247 -248 val_248 -249 val_249 -252 val_252 -255 val_255 -255 val_255 -256 val_256 -256 val_256 -257 val_257 -258 val_258 -26 val_26 -26 val_26 -260 val_260 -262 val_262 -263 val_263 -265 val_265 -265 val_265 -266 val_266 -27 val_27 -272 val_272 -272 val_272 -273 val_273 -273 val_273 -273 val_273 -274 val_274 -275 val_275 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -278 val_278 -278 val_278 -28 val_28 -280 val_280 -280 val_280 -281 val_281 -281 val_281 -282 val_282 -282 val_282 -283 val_283 -284 val_284 -285 val_285 -286 val_286 -287 val_287 -288 val_288 -288 val_288 -289 val_289 -291 val_291 -292 val_292 -296 val_296 -298 val_298 -298 val_298 -298 val_298 -30 val_30 -302 val_302 -305 val_305 -306 val_306 +163 val_163 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +10 val_10 +78 val_78 +483 val_483 +458 val_458 +458 val_458 +439 val_439 +439 val_439 +386 val_386 +332 val_332 307 val_307 307 val_307 -308 val_308 -309 val_309 -309 val_309 -310 val_310 -311 val_311 -311 val_311 -311 val_311 -315 val_315 -316 val_316 -316 val_316 -316 val_316 -317 val_317 -317 val_317 -318 val_318 -318 val_318 -318 val_318 -321 val_321 -321 val_321 -322 val_322 -322 val_322 -323 val_323 -325 val_325 -325 val_325 -327 val_327 -327 val_327 -327 val_327 -33 val_33 -331 val_331 -331 val_331 -332 val_332 -333 val_333 -333 val_333 -335 val_335 -336 val_336 -338 val_338 -339 val_339 -34 val_34 -341 val_341 -342 val_342 -342 val_342 -344 val_344 -344 val_344 -345 val_345 -348 val_348 -348 val_348 -348 val_348 -348 val_348 -348 val_348 +260 val_260 +235 val_235 +229 val_229 +229 val_229 +176 val_176 +176 val_176 +157 val_157 +496 val_496 +364 val_364 35 val_35 35 val_35 35 val_35 -351 val_351 -353 val_353 -353 val_353 -356 val_356 -360 val_360 -362 val_362 -364 val_364 -365 val_365 -366 val_366 -367 val_367 -367 val_367 -368 val_368 -369 val_369 -369 val_369 -369 val_369 -37 val_37 -37 val_37 -373 val_373 -374 val_374 -375 val_375 -377 val_377 -378 val_378 -379 val_379 -382 val_382 -382 val_382 -384 val_384 -384 val_384 -384 val_384 -386 val_386 -389 val_389 -392 val_392 -393 val_393 -394 val_394 -395 val_395 -395 val_395 -396 val_396 -396 val_396 -396 val_396 -397 val_397 -397 val_397 +345 val_345 +292 val_292 +286 val_286 +27 val_27 +189 val_189 +160 val_160 +116 val_116 +86 val_86 +477 val_477 +404 val_404 +404 val_404 399 val_399 399 val_399 -4 val_4 -400 val_400 -401 val_401 -401 val_401 -401 val_401 -401 val_401 +351 val_351 +273 val_273 +273 val_273 +273 val_273 +248 val_248 +200 val_200 +200 val_200 +195 val_195 +195 val_195 +19 val_19 +103 val_103 +103 val_103 +5 val_5 +5 val_5 +5 val_5 +493 val_493 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +449 val_449 401 val_401 -402 val_402 -403 val_403 -403 val_403 -403 val_403 -404 val_404 -404 val_404 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -407 val_407 -409 val_409 -409 val_409 -409 val_409 -41 val_41 -411 val_411 -413 val_413 -413 val_413 -414 val_414 -414 val_414 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +342 val_342 +342 val_342 +34 val_34 +323 val_323 +317 val_317 +317 val_317 +26 val_26 +26 val_26 +239 val_239 +239 val_239 +192 val_192 +186 val_186 +167 val_167 +167 val_167 +167 val_167 +113 val_113 +113 val_113 +77 val_77 +69 val_69 +51 val_51 +51 val_51 +436 val_436 +43 val_43 417 val_417 417 val_417 417 val_417 -418 val_418 -419 val_419 +339 val_339 +310 val_310 +213 val_213 +213 val_213 +207 val_207 +207 val_207 +129 val_129 +129 val_129 +84 val_84 +84 val_84 +452 val_452 +427 val_427 +374 val_374 +296 val_296 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +223 val_223 +223 val_223 +199 val_199 +199 val_199 +199 val_199 +170 val_170 +17 val_17 +145 val_145 +126 val_126 +67 val_67 +67 val_67 +490 val_490 +484 val_484 +459 val_459 +459 val_459 +41 val_41 +280 val_280 +280 val_280 +183 val_183 +177 val_177 +158 val_158 +76 val_76 +76 val_76 +487 val_487 42 val_42 42 val_42 -421 val_421 -424 val_424 -424 val_424 -427 val_427 -429 val_429 -429 val_429 -43 val_43 +414 val_414 +414 val_414 +336 val_336 +283 val_283 +258 val_258 +83 val_83 +83 val_83 +478 val_478 +478 val_478 430 val_430 430 val_430 430 val_430 -431 val_431 -431 val_431 -431 val_431 -432 val_432 -435 val_435 -436 val_436 -437 val_437 +411 val_411 +333 val_333 +333 val_333 +327 val_327 +327 val_327 +327 val_327 +308 val_308 +274 val_274 +255 val_255 +255 val_255 +249 val_249 +201 val_201 +196 val_196 +104 val_104 +104 val_104 +497 val_497 +424 val_424 +424 val_424 +24 val_24 +24 val_24 +82 val_82 +74 val_74 +481 val_481 +378 val_378 +305 val_305 +252 val_252 +174 val_174 +174 val_174 +149 val_149 +149 val_149 +482 val_482 +457 val_457 438 val_438 438 val_438 438 val_438 -439 val_439 -439 val_439 -44 val_44 -443 val_443 -444 val_444 -446 val_446 -448 val_448 -449 val_449 -452 val_452 -453 val_453 -454 val_454 -454 val_454 -454 val_454 -455 val_455 -457 val_457 -458 val_458 -458 val_458 -459 val_459 -459 val_459 -460 val_460 -462 val_462 -462 val_462 -463 val_463 -463 val_463 -466 val_466 -466 val_466 -466 val_466 -467 val_467 -468 val_468 -468 val_468 -468 val_468 -468 val_468 +379 val_379 +331 val_331 +331 val_331 +306 val_306 +228 val_228 +209 val_209 +209 val_209 +175 val_175 +175 val_175 +156 val_156 +90 val_90 +90 val_90 +90 val_90 +494 val_494 +475 val_475 469 val_469 469 val_469 469 val_469 469 val_469 469 val_469 -47 val_47 -470 val_470 +421 val_421 +402 val_402 +397 val_397 +397 val_397 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +187 val_187 +187 val_187 +187 val_187 +168 val_168 +15 val_15 +15 val_15 +120 val_120 +120 val_120 472 val_472 -475 val_475 -477 val_477 -478 val_478 -478 val_478 -479 val_479 -480 val_480 -480 val_480 -480 val_480 -481 val_481 -482 val_482 -483 val_483 -484 val_484 +453 val_453 +394 val_394 +375 val_375 +369 val_369 +369 val_369 +369 val_369 +321 val_321 +321 val_321 +302 val_302 +224 val_224 +224 val_224 +218 val_218 +190 val_190 +165 val_165 +165 val_165 +146 val_146 +146 val_146 +491 val_491 +466 val_466 +466 val_466 +466 val_466 +315 val_315 +30 val_30 +262 val_262 +237 val_237 +237 val_237 +111 val_111 +57 val_57 +362 val_362 +318 val_318 +318 val_318 +318 val_318 +284 val_284 +133 val_133 +114 val_114 +98 val_98 +98 val_98 +72 val_72 +72 val_72 +64 val_64 485 val_485 -487 val_487 +479 val_479 +431 val_431 +431 val_431 +431 val_431 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +353 val_353 +353 val_353 +309 val_309 +309 val_309 +281 val_281 +281 val_281 +275 val_275 +256 val_256 +256 val_256 +197 val_197 +197 val_197 +178 val_178 +105 val_105 489 val_489 489 val_489 489 val_489 489 val_489 -490 val_490 -491 val_491 -492 val_492 -492 val_492 -493 val_493 -494 val_494 -495 val_495 -496 val_496 -497 val_497 +344 val_344 +344 val_344 +338 val_338 +291 val_291 +285 val_285 +266 val_266 +20 val_20 +134 val_134 +134 val_134 +65 val_65 +409 val_409 +409 val_409 +409 val_409 +356 val_356 +278 val_278 +278 val_278 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +152 val_152 +152 val_152 +47 val_47 +463 val_463 +463 val_463 +444 val_444 +419 val_419 +366 val_366 +288 val_288 +288 val_288 +181 val_181 +162 val_162 +137 val_137 +137 val_137 +0 val_0 +0 val_0 +0 val_0 +80 val_80 +8 val_8 498 val_498 498 val_498 498 val_498 -5 val_5 -5 val_5 -5 val_5 -51 val_51 -51 val_51 -53 val_53 -54 val_54 -57 val_57 -58 val_58 -58 val_58 -64 val_64 -65 val_65 -66 val_66 -67 val_67 -67 val_67 -69 val_69 -70 val_70 -70 val_70 -70 val_70 -72 val_72 -72 val_72 -74 val_74 -76 val_76 -76 val_76 -77 val_77 -78 val_78 -8 val_8 -80 val_80 -82 val_82 -83 val_83 -83 val_83 -84 val_84 -84 val_84 -85 val_85 -86 val_86 -87 val_87 -9 val_9 -90 val_90 -90 val_90 -90 val_90 -92 val_92 -95 val_95 -95 val_95 -96 val_96 +4 val_4 +221 val_221 +221 val_221 +202 val_202 +2 val_2 +143 val_143 +118 val_118 +118 val_118 97 val_97 97 val_97 -98 val_98 -98 val_98 +495 val_495 +403 val_403 +403 val_403 +403 val_403 +325 val_325 +325 val_325 +272 val_272 +272 val_272 +247 val_247 +194 val_194 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +12 val_12 +12 val_12 PREHOOK: query: explain select count(*) as c from part as e where p_size + 100 not in ( select p_type from part where p_brand = e.p_brand) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -5555,7 +5597,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@t2_n0 POSTHOOK: Lineage: t2_n0.c1 SCRIPT [] -Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0) PREHOOK: type: QUERY PREHOOK: Input: default@t1_n0 @@ -5575,10 +5617,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -5591,9 +5633,10 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs Map 4 @@ -5601,6 +5644,16 @@ STAGE PLANS: TableScan alias: t2_n0 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: c1 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: c1 (type: int) outputColumnNames: c1 @@ -5614,16 +5667,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: c1 (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -5631,56 +5674,46 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 4 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 5 Data size: 28 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: 4 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint), _col2 (type: bigint) + sort order: + Statistics: Num rows: 5 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + 0 + 1 + outputColumnNames: _col0, _col2, _col3, _col4 Statistics: Num rows: 5 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Select Operator + expressions: _col0 (type: int), _col3 (type: bigint), _col4 (type: bigint), _col2 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col4 Statistics: Num rows: 5 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int) - outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Statistics: Num rows: 5 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -5698,6 +5731,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -5705,7 +5750,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0) PREHOOK: type: QUERY PREHOOK: Input: default@t1_n0 @@ -5735,12 +5780,12 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) - Reducer 5 <- Map 4 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) - Reducer 6 <- Reducer 5 (SIMPLE_EDGE) - Reducer 7 <- Reducer 5 (SIMPLE_EDGE) - Reducer 9 <- Map 8 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 9 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE) + Reducer 8 <- Reducer 6 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -5750,38 +5795,17 @@ STAGE PLANS: Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: c1 (type: int), c2 (type: char(100)) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col1 (type: char(100)) - sort order: + - Map-reduce partition columns: _col1 (type: char(100)) - Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 4 - Map Operator Tree: - TableScan - alias: t2_n0 - filterExpr: c1 is not null (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: c1 is not null (type: boolean) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: c1 (type: int) - outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: UDFToDouble(_col0) (type: double) - sort order: + - Map-reduce partition columns: UDFToDouble(_col0) (type: double) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: char(100)) + sort order: + + Map-reduce partition columns: _col1 (type: char(100)) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs - Map 8 + Map 4 Map Operator Tree: TableScan alias: t1_n0 @@ -5802,6 +5826,27 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs + Map 9 + Map Operator Tree: + TableScan + alias: t2_n0 + filterExpr: c1 is not null (type: boolean) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: c1 is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: c1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToDouble(_col0) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col0) (type: double) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 Execution mode: llap Reduce Operator Tree: @@ -5845,6 +5890,20 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: char(100)) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToDouble(_col0) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col0) (type: double) + Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: char(100)) + Reducer 6 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -5856,8 +5915,8 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - aggregations: count(), count(_col0) - keys: _col1 (type: char(100)) + aggregations: count(), count(_col1) + keys: _col0 (type: char(100)) mode: hash outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE @@ -5868,7 +5927,7 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint), _col2 (type: bigint) Group By Operator - keys: _col0 (type: int), _col1 (type: char(100)) + keys: _col1 (type: int), _col0 (type: char(100)) mode: hash outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE @@ -5877,7 +5936,7 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: int), _col1 (type: char(100)) Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 6 + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -5892,7 +5951,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: char(100)) Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint), _col2 (type: bigint) - Reducer 7 + Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -5910,20 +5969,6 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int), _col1 (type: char(100)) Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: boolean) - Reducer 9 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - keys: KEY._col0 (type: char(100)) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: UDFToDouble(_col0) (type: double) - sort order: + - Map-reduce partition columns: UDFToDouble(_col0) (type: double) - Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: char(100)) Stage: Stage-0 Fetch Operator @@ -6818,7 +6863,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@t_n0 #### A masked pattern was here #### 7 -Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select t_n0.i from t_n0 where t_n0.j NOT IN (select t1_n0.i from t_n0 t1_n0 ) PREHOOK: type: QUERY PREHOOK: Input: default@t_n0 @@ -6836,10 +6881,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -6852,9 +6897,11 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: int) + value expressions: _col0 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs Map 4 @@ -6862,6 +6909,16 @@ STAGE PLANS: TableScan alias: t1_n0 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: i (type: int) outputColumnNames: i @@ -6875,16 +6932,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: i (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -6892,56 +6939,46 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: int) - sort order: + - Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col2 (type: bigint), _col3 (type: bigint) + sort order: + Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col3 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col1 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + 0 + 1 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 Statistics: Num rows: 3 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col2 = 0L) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 Statistics: Num rows: 3 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int) - outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col2 = 0L) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 3 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE - 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 + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + 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 Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -6959,6 +6996,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -6966,7 +7015,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select t_n0.i from t_n0 where t_n0.j NOT IN (select t1_n0.i from t_n0 t1_n0 ) PREHOOK: type: QUERY PREHOOK: Input: default@t_n0 @@ -6977,7 +7026,7 @@ POSTHOOK: Input: default@t_n0 #### A masked pattern was here #### 1 4 -Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product PREHOOK: query: explain select t_n0.i from t_n0 where t_n0.i NOT IN (select t1_n0.i from t_n0 t1_n0 ) PREHOOK: type: QUERY PREHOOK: Input: default@t_n0 @@ -6995,25 +7044,36 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) - Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 5 <- Map 1 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) + Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) + Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan - alias: t_n0 + alias: t1_n0 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: i (type: int) outputColumnNames: _col0 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) Select Operator expressions: i (type: int) outputColumnNames: i @@ -7027,61 +7087,71 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: i (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Right Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 4 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: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint), _col2 (type: bigint) - Reducer 3 + sort order: + Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean), _col2 (type: int) + Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col4 Statistics: Num rows: 4 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: bigint), _col1 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col4 Statistics: Num rows: 4 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int) - outputColumnNames: _col0 - Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Filter Operator + predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Statistics: Num rows: 4 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - 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 - Reducer 4 + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -7093,24 +7163,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - keys: KEY._col0 (type: int) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) Stage: Stage-0 Fetch Operator @@ -7118,7 +7170,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product PREHOOK: query: select t_n0.i from t_n0 where t_n0.i NOT IN (select t1_n0.i from t_n0 t1_n0 ) PREHOOK: type: QUERY PREHOOK: Input: default@t_n0 @@ -7280,15 +7332,15 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2 - residual filter predicates: {(_col2 > _col1)} + residual filter predicates: {(_col0 > _col2)} Statistics: Num rows: 13833 Data size: 3721077 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: string), _col2 (type: string) - outputColumnNames: _col0, _col2 + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 13833 Data size: 3721077 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - aggregations: count(), count(_col0) - keys: _col2 (type: string) + aggregations: count(), count(_col1) + keys: _col0 (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 26750 Basic stats: COMPLETE Column stats: COMPLETE diff --git a/ql/src/test/results/clientpositive/llap/subquery_null_agg.q.out b/ql/src/test/results/clientpositive/llap/subquery_null_agg.q.out index fab963b1fd..4591ff8460 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_null_agg.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_null_agg.q.out @@ -6,8 +6,8 @@ POSTHOOK: query: CREATE TABLE table_7 (int_col INT) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@table_7 -Warning: Shuffle Join MERGEJOIN[28][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain SELECT (t1.int_col) * (t1.int_col) AS int_col @@ -59,10 +59,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 5 (CUSTOM_SIMPLE_EDGE) - Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 5 <- Map 1 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 4 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) + Reducer 4 <- Map 1 (SIMPLE_EDGE) + Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -79,27 +79,27 @@ STAGE PLANS: predicate: false (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Group By Operator - aggregations: count() + keys: true (type: boolean) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE - value expressions: _col0 (type: bigint) + key expressions: _col0 (type: boolean) + sort order: + + Map-reduce partition columns: _col0 (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Filter Operator predicate: false (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Group By Operator - keys: true (type: boolean) + aggregations: count() mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: boolean) - sort order: + - Map-reduce partition columns: _col0 (type: boolean) - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + value expressions: _col0 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -107,54 +107,46 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: 0 1 outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 9 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 1 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator sort order: - Statistics: Num rows: 1 Data size: 9 Basic stats: PARTIAL Column stats: NONE - value expressions: _col1 (type: bigint) + Statistics: Num rows: 1 Data size: 1 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: 0 1 outputColumnNames: _col1, _col2 Statistics: Num rows: 1 Data size: 10 Basic stats: PARTIAL Column stats: NONE - Filter Operator - predicate: ((_col1 = 0L) or _col2 is null) (type: boolean) + Select Operator + expressions: _col2 (type: bigint), _col1 (type: boolean) + outputColumnNames: _col1, _col2 Statistics: Num rows: 1 Data size: 10 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: null (type: void) - outputColumnNames: _col0 + Filter Operator + predicate: ((_col1 = 0L) or _col2 is null) (type: boolean) Statistics: Num rows: 1 Data size: 10 Basic stats: PARTIAL Column stats: NONE - File Output Operator - compressed: false + Select Operator + expressions: null (type: void) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 10 Basic stats: PARTIAL 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 + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 10 Basic stats: PARTIAL 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 Reducer 4 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -170,6 +162,18 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE value expressions: _col0 (type: boolean) + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + value expressions: _col0 (type: bigint) Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out index 166bd60ce2..c72e4b2097 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out @@ -206,7 +206,7 @@ POSTHOOK: Input: default@part_null_n0 78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith 85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull 86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully -Warning: Shuffle Join MERGEJOIN[18][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[18][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part where p_size > (select * from tempty_n0) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -217,7 +217,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@part POSTHOOK: Input: default@tempty_n0 #### A masked pattern was here #### -Warning: Shuffle Join MERGEJOIN[18][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[18][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where p_size > (select * from tempty_n0) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -237,26 +237,11 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Map 5 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) - Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Map 4 (XPROD_EDGE), Map 5 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: part - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 3 Map Operator Tree: TableScan alias: tempty_n0 @@ -274,7 +259,7 @@ STAGE PLANS: value expressions: _col0 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs - Map 5 + Map 4 Map Operator Tree: TableScan alias: tempty_n0 @@ -289,7 +274,38 @@ STAGE PLANS: value expressions: _col0 (type: char(2)) Execution mode: vectorized, llap LLAP IO: no inputs + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -300,11 +316,11 @@ STAGE PLANS: 0 1 2 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 - residual filter predicates: {(UDFToDouble(_col5) > UDFToDouble(_col10))} + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + residual filter predicates: {(UDFToDouble(_col7) > UDFToDouble(_col1))} Statistics: Num rows: 8 Data size: 5712 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + expressions: _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 8 Data size: 5712 Basic stats: PARTIAL Column stats: NONE File Output Operator @@ -314,22 +330,6 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 4 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE - Filter Operator - predicate: (sq_count_check(_col0) <= 1) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE - Select Operator - Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE Stage: Stage-0 Fetch Operator @@ -337,7 +337,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where p_name = (select p_name from part_null_n0 where p_name is null) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -357,30 +357,11 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Map 3 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) - Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Map 1 (XPROD_EDGE), Map 4 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: part - filterExpr: (p_name = null) (type: boolean) - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (p_name = null) (type: boolean) - Statistics: Num rows: 1 Data size: 619 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) - outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 3 Map Operator Tree: TableScan alias: part_null_n0 @@ -405,7 +386,42 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs + Map 4 + Map Operator Tree: + TableScan + alias: part + filterExpr: (p_name = null) (type: boolean) + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (p_name = null) (type: boolean) + Statistics: Num rows: 1 Data size: 619 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) + outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -416,10 +432,10 @@ STAGE PLANS: 0 1 2 - outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + outputColumnNames: _col2, _col4, _col5, _col6, _col7, _col8, _col9, _col10 Statistics: Num rows: 1 Data size: 959 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: int), null (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + expressions: _col2 (type: int), null (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 1 Data size: 959 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -429,22 +445,6 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 4 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: (sq_count_check(_col0) <= 1) (type: boolean) - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE - Select Operator - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 Fetch Operator @@ -452,7 +452,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select * from part where p_name = (select p_name from part_null_n0 where p_name is null) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -463,8 +463,8 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@part POSTHOOK: Input: default@part_null_n0 #### A masked pattern was here #### -Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product PREHOOK: query: explain select * from part where (select i from tnull_n0 limit 1) is null PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -484,29 +484,14 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE) - Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Map 7 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE) + Reducer 5 <- Map 8 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) + Reducer 7 <- Map 6 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: part - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 4 Map Operator Tree: TableScan alias: tnull_n0 @@ -521,7 +506,7 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs - Map 7 + Map 6 Map Operator Tree: TableScan alias: tnull_n0 @@ -540,34 +525,80 @@ STAGE PLANS: value expressions: _col0 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs + Map 8 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + 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: COMPLETE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: 0 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator sort order: - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - Reducer 3 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 5 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: 0 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: int) + expressions: _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string), _col1 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator @@ -584,38 +615,7 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Limit - Number of rows: 1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 6 - Execution mode: vectorized, llap - 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: COMPLETE - Filter Operator - predicate: (sq_count_check(_col0) <= 1) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 8 + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Select Operator @@ -636,8 +636,8 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product PREHOOK: query: select * from part where (select i from tnull_n0 limit 1) is null PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -980,7 +980,7 @@ POSTHOOK: Input: default@part 42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl 49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick 90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl -Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product PREHOOK: query: explain select p_mfgr, p_name, p_size from part where part.p_size > (select first_value(p_size) over(partition by p_mfgr order by p_size) as fv from part order by fv limit 1) @@ -1002,19 +1002,31 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 4 <- Map 3 (SIMPLE_EDGE) - Reducer 5 <- Reducer 4 (SIMPLE_EDGE) - Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) - Reducer 7 <- Reducer 4 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE) + Reducer 5 <- Map 7 (XPROD_EDGE), Reducer 4 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 6 <- Reducer 2 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan alias: part - Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator + Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_size (type: int) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE @@ -1024,44 +1036,7 @@ STAGE PLANS: value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs - Map 3 - Map Operator Tree: - TableScan - alias: part - Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: p_mfgr (type: string), p_size (type: int) - sort order: ++ - Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized, llap - LLAP IO: no inputs Reducer 2 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - Inner Join 0 to 2 - keys: - 0 - 1 - 2 - outputColumnNames: _col0, _col1, _col2, _col4 - residual filter predicates: {(_col2 > _col4)} - Statistics: Num rows: 8 Data size: 1816 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col1 (type: string), _col0 (type: string), _col2 (type: int) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 8 Data size: 1784 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 8 Data size: 1784 Basic stats: COMPLETE Column stats: COMPLETE - 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 - Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: Select Operator @@ -1125,7 +1100,7 @@ STAGE PLANS: sort order: + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 - Reducer 5 + Reducer 3 Execution mode: vectorized, llap Reduce Operator Tree: Select Operator @@ -1142,7 +1117,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 6 + Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -1158,7 +1133,32 @@ STAGE PLANS: Reduce Output Operator sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 7 + Reducer 5 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 + 1 + 2 + outputColumnNames: _col1, _col2, _col3, _col4 + residual filter predicates: {(_col4 > _col1)} + Statistics: Num rows: 8 Data size: 1816 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: string), _col2 (type: string), _col4 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 1784 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 1784 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Select Operator @@ -1179,7 +1179,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product PREHOOK: query: select p_mfgr, p_name, p_size from part where part.p_size > (select first_value(p_size) over(partition by p_mfgr order by p_size) as fv from part order by fv limit 1) @@ -2817,7 +2817,7 @@ POSTHOOK: Input: default@part #### A masked pattern was here #### 2346.3 3461.37 -Warning: Shuffle Join MERGEJOIN[76][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[76][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 6' is a cross product PREHOOK: query: explain select * from part where p_size > (select count(p_name) from part INTERSECT select count(p_brand) from part) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2835,34 +2835,39 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 11 <- Union 10 (SIMPLE_EDGE) - Reducer 13 <- Map 12 (CUSTOM_SIMPLE_EDGE) - Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Union 6 (CONTAINS) - Reducer 15 <- Reducer 13 (SIMPLE_EDGE), Union 10 (CONTAINS) - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 11 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) - Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE) - Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS) - Reducer 7 <- Union 6 (SIMPLE_EDGE) - Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE) - Reducer 9 <- Reducer 4 (SIMPLE_EDGE), Union 10 (CONTAINS) + Reducer 10 <- Reducer 9 (CUSTOM_SIMPLE_EDGE) + Reducer 12 <- Map 11 (CUSTOM_SIMPLE_EDGE) + Reducer 13 <- Reducer 12 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 14 <- Reducer 12 (SIMPLE_EDGE), Union 8 (CONTAINS) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 5 <- Union 4 (SIMPLE_EDGE) + Reducer 6 <- Map 15 (XPROD_EDGE), Reducer 10 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) + Reducer 7 <- Reducer 2 (SIMPLE_EDGE), Union 8 (CONTAINS) + Reducer 9 <- Union 8 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan alias: part - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(p_name) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs - Map 12 + Map 11 Map Operator Tree: TableScan alias: part @@ -2882,47 +2887,38 @@ STAGE PLANS: value expressions: _col0 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs - Map 3 + Map 15 Map Operator Tree: TableScan alias: part - Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: p_name (type: string) - outputColumnNames: p_name - Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(p_name) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) + expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs - Reducer 11 + Reducer 10 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: bigint) mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (_col1 = 2L) (type: boolean) - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 13 + Reducer 12 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -2948,7 +2944,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: bigint) Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint) - Reducer 14 + Reducer 13 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -2969,7 +2965,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: bigint) Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint) - Reducer 15 + Reducer 14 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -2991,31 +2987,6 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint) Reducer 2 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - Inner Join 0 to 2 - keys: - 0 - 1 - 2 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 - residual filter predicates: {(UDFToLong(_col5) > _col10)} - Statistics: Num rows: 8 Data size: 5016 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 8 Data size: 4952 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 8 Data size: 4952 Basic stats: COMPLETE Column stats: COMPLETE - 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 - Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3041,7 +3012,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: bigint) Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint) - Reducer 5 + Reducer 3 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3062,7 +3033,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: bigint) Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint) - Reducer 7 + Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3071,41 +3042,43 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col1 (type: bigint) - outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (_col1 = 2L) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 8 - Execution mode: vectorized, llap - 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: COMPLETE Filter Operator - predicate: (sq_count_check(_col0) <= 1) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (_col1 = 2L) (type: boolean) + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 9 + value expressions: _col0 (type: bigint) + Reducer 6 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 + 1 + 2 + outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + residual filter predicates: {(UDFToLong(_col7) > _col0)} + Statistics: Num rows: 8 Data size: 5016 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 8 Data size: 4952 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 4952 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3126,10 +3099,37 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: bigint) Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint) - Union 10 - Vertex: Union 10 - Union 6 - Vertex: Union 6 + Reducer 9 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col1 = 2L) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Union 4 + Vertex: Union 4 + Union 8 + Vertex: Union 8 Stage: Stage-0 Fetch Operator @@ -3137,7 +3137,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[76][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[76][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 6' is a cross product PREHOOK: query: select * from part where p_size > (select count(p_name) from part INTERSECT select count(p_brand) from part) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -3340,7 +3340,8 @@ POSTHOOK: Input: default@part 85768 86428 90681 -Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product PREHOOK: query: explain select * from part_null_n0 where p_name NOT LIKE (select min(p_name) from part_null_n0) AND p_brand NOT IN (select p_name from part) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -3360,29 +3361,62 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) - Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE) - Reducer 7 <- Map 6 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Map 6 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) + Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 9 <- Map 8 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan - alias: part_null_n0 - Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment_n11 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + alias: part + Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(p_name) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 7 + Map Operator Tree: + TableScan + alias: part_null_n0 + Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment_n11 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - sort order: + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs - Map 4 + Map 8 Map Operator Tree: TableScan alias: part_null_n0 @@ -3402,94 +3436,70 @@ STAGE PLANS: value expressions: _col0 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs - Map 6 - Map Operator Tree: - TableScan - alias: part - Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: p_name (type: string) - outputColumnNames: p_name - Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(p_name) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: p_name (type: string) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized, llap - LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 - Inner Join 0 to 2 + Right Outer Join 0 to 1 keys: - 0 - 1 - 2 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 - residual filter predicates: {(not (_col1 like _col9))} - Statistics: Num rows: 1 Data size: 1505 Basic stats: COMPLETE Column stats: NONE + 0 _col0 (type: string) + 1 _col3 (type: string) + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 14 Data size: 1787 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col3 (type: string) - sort order: + - Map-reduce partition columns: _col3 (type: string) - Statistics: Num rows: 1 Data size: 1505 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: bigint), _col11 (type: bigint) - Reducer 3 + sort order: + Statistics: Num rows: 14 Data size: 1787 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string) + Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col3 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13 - Statistics: Num rows: 14 Data size: 1787 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col10 = 0L) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) - Statistics: Num rows: 9 Data size: 1148 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 9 Data size: 1148 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 9 Data size: 1148 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 + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13 + residual filter predicates: {(not (_col3 like _col13))} + Statistics: Num rows: 7 Data size: 3595 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col1 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13 + Statistics: Num rows: 7 Data size: 3595 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col10 = 0L) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) + Statistics: Num rows: 4 Data size: 2054 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 4 Data size: 2054 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 2054 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 Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: min(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: string) - Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3501,24 +3511,33 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 8 + Reducer 6 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 385 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 385 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: string) + Reducer 9 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator - keys: KEY._col0 (type: string) + aggregations: min(VALUE._col0) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) + Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) Stage: Stage-0 Fetch Operator @@ -3526,7 +3545,8 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product PREHOOK: query: select * from part_null_n0 where p_name NOT LIKE (select min(p_name) from part_null_n0) AND p_brand NOT IN (select p_name from part) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -3562,7 +3582,7 @@ POSTHOOK: Input: default@part_null_n0 85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull 86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully 90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl -Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product PREHOOK: query: explain select * from part_null_n0 where p_brand NOT IN (select p_name from part) AND p_name NOT LIKE (select min(p_name) from part_null_n0 pp where part_null_n0.p_type = pp.p_type) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -3582,38 +3602,29 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) - Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE) - Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE) - Reducer 7 <- Map 5 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) + Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE) + Reducer 6 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 9 <- Map 8 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: part_null_n0 - filterExpr: p_type is not null (type: boolean) - Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: p_type is not null (type: boolean) - Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment_n11 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 5 Map Operator Tree: TableScan alias: part Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: p_name (type: string) outputColumnNames: p_name @@ -3627,16 +3638,27 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: p_name (type: string) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 7 + Map Operator Tree: + TableScan + alias: part_null_n0 + filterExpr: p_type is not null (type: boolean) + Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment_n11 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Map 8 @@ -3663,47 +3685,67 @@ STAGE PLANS: Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Right Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 1 Data size: 1137 Basic stats: COMPLETE Column stats: NONE + 0 _col0 (type: string) + 1 _col3 (type: string) + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 14 Data size: 1787 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col3 (type: string) - sort order: + - Map-reduce partition columns: _col3 (type: string) - Statistics: Num rows: 1 Data size: 1137 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: bigint) - Reducer 3 + sort order: + Statistics: Num rows: 14 Data size: 1787 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string) + Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col3 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 - Statistics: Num rows: 14 Data size: 1787 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col12 is null and _col3 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) - Statistics: Num rows: 9 Data size: 1148 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 9 Data size: 1148 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col4 (type: string) - sort order: + - Map-reduce partition columns: _col4 (type: string) - Statistics: Num rows: 9 Data size: 1148 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) - Reducer 4 + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12 + Statistics: Num rows: 14 Data size: 2025 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string), _col11 (type: bigint), _col12 (type: bigint), _col1 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + Statistics: Num rows: 14 Data size: 2025 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col12 is null and _col3 is not null and (_col10 >= _col9)) or (_col9 = 0L)) (type: boolean) + Statistics: Num rows: 9 Data size: 1301 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 9 Data size: 1301 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: string) + sort order: + + Map-reduce partition columns: _col4 (type: string) + Statistics: Num rows: 9 Data size: 1301 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Reducer 5 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -3714,14 +3756,14 @@ STAGE PLANS: 1 _col1 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col13 residual filter predicates: {(not (_col1 like _col13))} - Statistics: Num rows: 5 Data size: 701 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 795 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 5 Data size: 701 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 795 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 701 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 795 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -3738,24 +3780,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 7 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - keys: KEY._col0 (type: string) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) Reducer 9 Execution mode: vectorized, llap Reduce Operator Tree: @@ -3782,7 +3806,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product PREHOOK: query: select * from part_null_n0 where p_brand NOT IN (select p_name from part) AND p_name NOT LIKE (select min(p_name) from part_null_n0 pp where part_null_n0.p_type = pp.p_type) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -3794,7 +3818,7 @@ POSTHOOK: Input: default@part POSTHOOK: Input: default@part_null_n0 #### A masked pattern was here #### 90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl -Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select p.p_partkey, li.l_suppkey from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey where li.l_linenumber = 1 and @@ -3818,34 +3842,13 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) - Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) - Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) Reducer 7 <- Map 6 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: lineitem - filterExpr: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: l_partkey (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 5 Map Operator Tree: TableScan alias: li @@ -3866,6 +3869,27 @@ STAGE PLANS: value expressions: _col0 (type: int), _col2 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs + Map 4 + Map Operator Tree: + TableScan + alias: lineitem + filterExpr: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs Map 6 Map Operator Tree: TableScan @@ -3894,34 +3918,21 @@ STAGE PLANS: Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - keys: KEY._col0 (type: int) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col1 (type: int) - outputColumnNames: _col0, _col1, _col3 + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator sort order: Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: int), _col3 (type: int) - Reducer 4 + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -3930,11 +3941,11 @@ STAGE PLANS: keys: 0 1 - outputColumnNames: _col0, _col1, _col3, _col5 - residual filter predicates: {(_col1 <> _col5)} + outputColumnNames: _col0, _col1, _col2, _col5 + residual filter predicates: {(_col0 <> _col5)} Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col3 (type: int) + expressions: _col1 (type: int), _col2 (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 14 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -3944,6 +3955,19 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: @@ -3968,7 +3992,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select p.p_partkey, li.l_suppkey from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey where li.l_linenumber = 1 and @@ -4008,7 +4032,7 @@ POSTHOOK: Input: default@lineitem 85951 5952 88035 5560 88362 871 -Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select p.p_partkey, li.l_suppkey from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey where li.l_linenumber = 1 and @@ -4032,34 +4056,13 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) - Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) - Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) Reducer 7 <- Map 6 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: lineitem - filterExpr: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: l_partkey (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 5 Map Operator Tree: TableScan alias: li @@ -4080,6 +4083,27 @@ STAGE PLANS: value expressions: _col0 (type: int), _col2 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs + Map 4 + Map Operator Tree: + TableScan + alias: lineitem + filterExpr: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs Map 6 Map Operator Tree: TableScan @@ -4108,34 +4132,21 @@ STAGE PLANS: Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - keys: KEY._col0 (type: int) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col1 (type: int) - outputColumnNames: _col0, _col1, _col3 + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator sort order: Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: int), _col3 (type: int) - Reducer 4 + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -4144,11 +4155,11 @@ STAGE PLANS: keys: 0 1 - outputColumnNames: _col0, _col1, _col3, _col5 - residual filter predicates: {(_col1 <> _col5)} + outputColumnNames: _col0, _col1, _col2, _col5 + residual filter predicates: {(_col0 <> _col5)} Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col3 (type: int) + expressions: _col1 (type: int), _col2 (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 14 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -4158,6 +4169,19 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 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: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: @@ -4182,7 +4206,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select p.p_partkey, li.l_suppkey from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey where li.l_linenumber = 1 and @@ -5511,7 +5535,7 @@ POSTHOOK: Input: default@src 468 4 469 5 489 4 -Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[46][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product PREHOOK: query: explain select key, value, count(*) from src b @@ -5539,34 +5563,14 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE) - Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) - Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 4 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 6 <- Map 1 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: b - filterExpr: (key > '8') (type: boolean) - Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (key > '8') (type: boolean) - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 5 Map Operator Tree: TableScan alias: src @@ -5575,20 +5579,16 @@ STAGE PLANS: Filter Operator predicate: (key > '8') (type: boolean) Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: key (type: string) + Group By Operator + keys: key (type: string) + mode: hash outputColumnNames: _col0 - Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (key > '9') (type: boolean) Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE @@ -5605,20 +5605,54 @@ STAGE PLANS: value expressions: _col0 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs + Map 7 + Map Operator Tree: + TableScan + alias: b + filterExpr: (key > '8') (type: boolean) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '8') (type: boolean) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Semi Join 0 to 1 + Inner Join 0 to 1 keys: 0 _col0 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1 + outputColumnNames: _col1, _col2 Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() - keys: _col0 (type: string), _col1 (type: string) + keys: _col1 (type: string), _col2 (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE @@ -5628,7 +5662,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: bigint) - Reducer 3 + Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -5641,7 +5675,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint) - Reducer 4 + Reducer 5 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -5683,7 +5717,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[46][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product PREHOOK: query: select key, value, count(*) from src b where b.key in (select key from src where src.key > '8') @@ -5700,7 +5734,7 @@ having count(*) > (select count(*) from src s1 where s1.key > '9' ) POSTHOOK: type: QUERY POSTHOOK: Input: default@src #### A masked pattern was here #### -Warning: Shuffle Join MERGEJOIN[27][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[27][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select * from part where p_size > (select max(p_size) from part group by p_type) PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -5718,28 +5752,36 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 4 <- Map 3 (SIMPLE_EDGE) - Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE) - Reducer 7 <- Map 6 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 7 (XPROD_EDGE), Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan alias: part - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: p_type, p_size + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs - Map 3 + Map 4 Map Operator Tree: TableScan alias: part @@ -5760,30 +5802,39 @@ STAGE PLANS: Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs - Map 6 + Map 7 Map Operator Tree: TableScan alias: part - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: p_type (type: string), p_size (type: int) - outputColumnNames: p_type, p_size - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: max(p_size) - keys: p_type (type: string) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int) + expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -5794,11 +5845,11 @@ STAGE PLANS: 0 1 2 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 - residual filter predicates: {(_col5 > _col10)} + outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + residual filter predicates: {(_col7 > _col0)} Statistics: Num rows: 112 Data size: 69776 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + expressions: _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: int), _col8 (type: string), _col9 (type: double), _col10 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 112 Data size: 69328 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -5808,7 +5859,7 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 4 + Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -5827,7 +5878,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 5 + Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -5843,23 +5894,6 @@ STAGE PLANS: Reduce Output Operator sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 7 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: max(VALUE._col0) - keys: KEY._col0 (type: string) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col1 (type: int) - outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/llap/subquery_select.q.out b/ql/src/test/results/clientpositive/llap/subquery_select.q.out index 854d215a23..aca143896a 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_select.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_select.q.out @@ -1,4 +1,4 @@ -Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain SELECT p_size, p_size IN ( SELECT MAX(p_size) FROM part) FROM part @@ -20,8 +20,8 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### @@ -36,9 +36,10 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) Select Operator expressions: p_size (type: int) outputColumnNames: p_size @@ -72,31 +73,29 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 26 Data size: 520 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 27 Data size: 116 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: 26 Data size: 520 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint), _col2 (type: bigint) + sort order: + Statistics: Num rows: 27 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + 0 + 1 + outputColumnNames: _col0, _col2, _col3, _col4 Statistics: Num rows: 27 Data size: 548 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), CASE WHEN ((_col1 = 0L)) THEN (false) WHEN (_col4 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col2 < _col1)) THEN (null) ELSE (false) END (type: boolean) + expressions: _col0 (type: int), CASE WHEN ((_col3 = 0L)) THEN (false) WHEN (_col2 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (null) ELSE (false) END (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 27 Data size: 216 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -114,15 +113,16 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col0) - mode: complete + Select Operator + expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + 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: COMPLETE + value expressions: _col1 (type: boolean) Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: @@ -131,16 +131,15 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), true (type: boolean) + Group By Operator + aggregations: count(), count(_col0) + mode: complete outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -148,7 +147,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: SELECT p_size, p_size IN ( SELECT MAX(p_size) FROM part) FROM part @@ -161,32 +160,32 @@ FROM part POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### +46 true 1 false +31 false +34 false +7 false +39 false +6 false +6 false 2 false 2 false 2 false 2 false -6 false -6 false -7 false -10 false -12 false +45 false 14 false 14 false +12 false +42 false +10 false +40 false 17 false -18 false -19 false -23 false -25 false -27 false 28 false -31 false -34 false -39 false -40 false -42 false -45 false -46 true +27 false +25 false +23 false +19 false +18 false PREHOOK: query: EXPLAIN SELECT p_size, p_size IN ( SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) FROM part @@ -388,7 +387,7 @@ POSTHOOK: Input: default@part 6 true 18 true 45 true -Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain SELECT p_size, p_size NOT IN ( SELECT MAX(p_size) FROM part) FROM part @@ -410,8 +409,8 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### @@ -426,9 +425,10 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) Select Operator expressions: p_size (type: int) outputColumnNames: p_size @@ -462,31 +462,29 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 26 Data size: 520 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 27 Data size: 116 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: 26 Data size: 520 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint), _col2 (type: bigint) + sort order: + Statistics: Num rows: 27 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + 0 + 1 + outputColumnNames: _col0, _col2, _col3, _col4 Statistics: Num rows: 27 Data size: 548 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), CASE WHEN ((_col1 = 0L)) THEN (true) WHEN (_col4 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col2 < _col1)) THEN (null) ELSE (true) END (type: boolean) + expressions: _col0 (type: int), CASE WHEN ((_col3 = 0L)) THEN (true) WHEN (_col2 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (null) ELSE (true) END (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 27 Data size: 216 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -504,15 +502,16 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col0) - mode: complete + Select Operator + expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + 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: COMPLETE + value expressions: _col1 (type: boolean) Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: @@ -521,16 +520,15 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), true (type: boolean) + Group By Operator + aggregations: count(), count(_col0) + mode: complete outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -538,7 +536,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: SELECT p_size, p_size NOT IN ( SELECT MAX(p_size) FROM part) FROM part @@ -551,32 +549,32 @@ FROM part POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### -1 true +10 true 2 true 2 true 2 true 2 true -6 true -6 true -7 true -10 true -12 true +46 false +1 true +34 true +45 true 14 true 14 true -17 true -18 true -19 true -23 true -25 true -27 true -28 true 31 true -34 true +12 true 39 true -40 true 42 true -45 true -46 false +7 true +25 true +18 true +6 true +6 true +17 true +40 true +27 true +28 true +19 true +23 true PREHOOK: query: EXPLAIN SELECT p_size, p_size NOT IN ( SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) FROM part @@ -598,44 +596,41 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) - Reducer 3 <- Reducer 2 (ONE_TO_ONE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) - Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) - Reducer 5 <- Map 1 (SIMPLE_EDGE) - Reducer 6 <- Map 1 (SIMPLE_EDGE) - Reducer 8 <- Map 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE) + Reducer 4 <- Reducer 3 (ONE_TO_ONE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) + Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 7 <- Map 6 (SIMPLE_EDGE) + Reducer 8 <- Map 6 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan - alias: part - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: p_type (type: string), p_size (type: int) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int) + alias: p + filterExpr: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: p_type is not null (type: boolean) - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - aggregations: max(p_size) keys: p_type (type: string) mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int) + Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 6 + Map Operator Tree: + TableScan + alias: p + filterExpr: (p_type is not null or p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: p_type is not null (type: boolean) Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE @@ -651,30 +646,54 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: int) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 7 - Map Operator Tree: - TableScan - alias: p - filterExpr: p_type is not null (type: boolean) - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) Filter Operator predicate: p_type is not null (type: boolean) - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator + aggregations: max(p_size) keys: p_type (type: string) mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -698,7 +717,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: int) - Reducer 3 + Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -710,20 +729,20 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col3, _col4 Statistics: Num rows: 13 Data size: 1612 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: int) + key expressions: _col1 (type: int), _col0 (type: string) sort order: ++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: int) + Map-reduce partition columns: _col1 (type: int), _col0 (type: string) Statistics: Num rows: 13 Data size: 1612 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col3 (type: bigint), _col4 (type: bigint) - Reducer 4 + Reducer 5 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: Left Outer Join 0 to 1 keys: - 0 _col0 (type: string), _col1 (type: int) - 1 _col1 (type: string), _col0 (type: int) + 0 _col1 (type: int), _col0 (type: string) + 1 _col0 (type: int), _col1 (type: string) outputColumnNames: _col1, _col3, _col4, _col7 Statistics: Num rows: 13 Data size: 312 Basic stats: COMPLETE Column stats: COMPLETE Select Operator @@ -737,28 +756,7 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: max(VALUE._col0) - keys: KEY._col0 (type: string) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col1) - keys: _col0 (type: string) - mode: complete - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 13 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint), _col2 (type: bigint) - Reducer 6 + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -775,31 +773,32 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: string), _col0 (type: int) + key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Map-reduce partition columns: _col1 (type: string), _col0 (type: int) + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: boolean) Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator + aggregations: max(VALUE._col0) keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - aggregations: count() + aggregations: count(), count(_col1) keys: _col0 (type: string) mode: complete - outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint) + Statistics: Num rows: 13 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint), _col2 (type: bigint) Stage: Stage-0 Fetch Operator @@ -819,32 +818,32 @@ FROM part POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### -7 false -46 false 1 false -40 false -23 false -34 false -42 false -14 false -19 false 2 false -17 true -31 false -14 false -6 false 2 false 2 false -39 false -10 false 2 false -27 false -25 false -28 false -12 false 6 false +6 false +7 false +10 false +12 false +14 false +14 false +17 true 18 false +19 false +23 false +25 false +27 false +28 false +31 false +34 false +39 false +40 false +42 false 45 false +46 false Warning: Shuffle Join MERGEJOIN[14][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: explain SELECT p_size, EXISTS(SELECT p_size FROM part) FROM part @@ -2024,16 +2023,16 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) - Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE) - Reducer 4 <- Map 1 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 3 <- Map 1 (SIMPLE_EDGE) + Reducer 4 <- Map 1 (SIMPLE_EDGE), Reducer 3 (ONE_TO_ONE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan alias: b - filterExpr: ((value is not null and key is not null) or ((key > '9') and value is not null) or value is not null) (type: boolean) + filterExpr: ((value is not null and key is not null) or value is not null or ((key > '9') and value is not null)) (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (key is not null and value is not null) (type: boolean) @@ -2047,18 +2046,6 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((key > '9') and value is not null) (type: boolean) - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: value (type: string) - outputColumnNames: _col1 - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col1 (type: string) - sort order: + - Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: value is not null (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE @@ -2074,6 +2061,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string) + Filter Operator + predicate: ((key > '9') and value is not null) (type: boolean) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -2095,31 +2094,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Reducer 3 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col1 (type: string) - 1 _col1 (type: string) - outputColumnNames: _col1, _col2 - Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col2 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: _col0 (type: string), _col1 (type: string) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 83 Data size: 22825 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: string) - sort order: ++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 83 Data size: 22825 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -2141,6 +2115,31 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 250 Data size: 68750 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string) + Reducer 4 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col1 (type: string) + outputColumnNames: _col0, _col3 + Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 22825 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 83 Data size: 22825 Basic stats: COMPLETE Column stats: COMPLETE Stage: Stage-0 Fetch Operator @@ -2194,7 +2193,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tnull_n3 POSTHOOK: Lineage: tnull_n3.i EXPRESSION [] -Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select p_size, p_size IN (select i from tnull_n3) from part PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2214,10 +2213,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (ONE_TO_ONE_EDGE) - Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -2230,9 +2229,10 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs Map 4 @@ -2240,6 +2240,16 @@ STAGE PLANS: TableScan alias: tnull_n3 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: i (type: int) outputColumnNames: i @@ -2253,16 +2263,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - keys: i (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 @@ -2270,31 +2270,29 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 26 Data size: 520 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 27 Data size: 116 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: 26 Data size: 520 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint), _col2 (type: bigint) + sort order: + Statistics: Num rows: 27 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + 0 + 1 + outputColumnNames: _col0, _col2, _col3, _col4 Statistics: Num rows: 27 Data size: 548 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), CASE WHEN ((_col1 = 0L)) THEN (false) WHEN (_col4 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col2 < _col1)) THEN (null) ELSE (false) END (type: boolean) + expressions: _col0 (type: int), CASE WHEN ((_col3 = 0L)) THEN (false) WHEN (_col2 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (null) ELSE (false) END (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 27 Data size: 216 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -2305,18 +2303,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Reducer 5 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -2334,6 +2320,18 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -2341,7 +2339,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select p_size, p_size IN (select i from tnull_n3) from part PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2353,31 +2351,31 @@ POSTHOOK: Input: default@part POSTHOOK: Input: default@tnull_n3 #### A masked pattern was here #### 1 NULL -2 NULL -2 NULL -2 NULL -2 NULL -6 NULL -6 NULL -7 NULL 10 NULL -12 NULL -14 NULL -14 NULL 17 NULL -18 NULL -19 NULL -23 NULL -25 NULL -27 NULL -28 NULL 31 NULL -34 NULL +7 NULL +28 NULL 39 NULL +6 NULL +6 NULL 40 NULL 42 NULL +34 NULL +23 NULL +25 NULL +2 NULL +2 NULL +2 NULL +2 NULL 45 NULL +14 NULL +14 NULL 46 NULL +12 NULL +27 NULL +19 NULL +18 NULL PREHOOK: query: CREATE TABLE tempty_n3(i int) PREHOOK: type: CREATETABLE PREHOOK: Output: database:default @@ -2944,8 +2942,8 @@ POSTHOOK: Input: default@part 6 false 18 false 45 false -Warning: Shuffle Join MERGEJOIN[31][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[31][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product PREHOOK: query: explain select p_type, (select p_size from part order by p_size limit 1) = 1 from part order by p_type PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -2963,30 +2961,15 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE) - Reducer 4 <- Reducer 3 (SIMPLE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE) - Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Map 5 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE) + Reducer 5 <- Map 8 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) + Reducer 6 <- Reducer 5 (SIMPLE_EDGE) + Reducer 7 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: part - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: p_type (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 5 Map Operator Tree: TableScan alias: part @@ -3006,34 +2989,82 @@ STAGE PLANS: TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap LLAP IO: no inputs + Map 8 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_type (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + 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: COMPLETE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: 0 1 - outputColumnNames: _col0 - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator sort order: - Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string) - Reducer 3 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 5 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: 0 1 - outputColumnNames: _col0, _col2 + outputColumnNames: _col1, _col2 Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: string), (_col2 = 1) (type: boolean) + expressions: _col2 (type: string), (_col1 = 1) (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator @@ -3041,7 +3072,7 @@ STAGE PLANS: sort order: + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - Reducer 4 + Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Select Operator @@ -3055,40 +3086,7 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 6 - Execution mode: vectorized, llap - Reduce Operator Tree: - Select Operator - Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE - Limit - Number of rows: 1 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Reducer 7 - Execution mode: vectorized, llap - 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: COMPLETE - Filter Operator - predicate: (sq_count_check(_col0) <= 1) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Select Operator @@ -3109,8 +3107,8 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[31][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[31][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product PREHOOK: query: select p_type, (select p_size from part order by p_size limit 1) = 1 from part order by p_type PREHOOK: type: QUERY PREHOOK: Input: default@part @@ -3145,7 +3143,7 @@ STANDARD ANODIZED TIN true STANDARD BURNISHED TIN true STANDARD PLATED TIN true STANDARD POLISHED STEEL true -Warning: Shuffle Join MERGEJOIN[72][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[73][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product PREHOOK: query: EXPLAIN SELECT p_size, p_size IN ( SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) AND p_name IN (SELECT min(p_name) from part) @@ -3172,8 +3170,8 @@ STAGE PLANS: Reducer 10 <- Map 9 (CUSTOM_SIMPLE_EDGE) Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE) Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) - Reducer 4 <- Reducer 10 (XPROD_EDGE), Reducer 3 (XPROD_EDGE) - Reducer 5 <- Reducer 10 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 4 <- Reducer 10 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 10 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) Reducer 7 <- Map 6 (SIMPLE_EDGE) Reducer 8 <- Map 6 (SIMPLE_EDGE) #### A masked pattern was here #### @@ -3252,15 +3250,6 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(_col0) - mode: complete - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) Select Operator expressions: _col0 (type: string), true (type: boolean) outputColumnNames: _col0, _col1 @@ -3271,6 +3260,15 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Group By Operator + aggregations: count(), count(_col0) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 2 Execution mode: llap Reduce Operator Tree: @@ -3298,29 +3296,14 @@ STAGE PLANS: 0 _col1 (type: string), _col2 (type: int) 1 _col1 (type: string), _col0 (type: int) outputColumnNames: _col0, _col2, _col4, _col5, _col8 - Statistics: Num rows: 48 Data size: 6468 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 48 Data size: 6468 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col2 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col8 (type: boolean) - Reducer 4 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 - 1 - outputColumnNames: _col0, _col2, _col4, _col5, _col8, _col9, _col10 - Statistics: Num rows: 48 Data size: 7236 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 6468 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 48 Data size: 7236 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col8 (type: boolean), _col9 (type: bigint), _col10 (type: bigint) - Reducer 5 + Statistics: Num rows: 48 Data size: 6468 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col8 (type: boolean) + Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -3329,10 +3312,25 @@ STAGE PLANS: keys: 0 _col0 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col2, _col4, _col5, _col8, _col9, _col10, _col12 + outputColumnNames: _col0, _col2, _col4, _col5, _col8, _col10 + Statistics: Num rows: 49 Data size: 6621 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 49 Data size: 6621 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col2 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col8 (type: boolean), _col10 (type: boolean) + Reducer 5 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col2, _col4, _col5, _col8, _col10, _col11, _col12 Statistics: Num rows: 49 Data size: 7405 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col2 (type: int), (CASE WHEN ((_col4 = 0L)) THEN (false) WHEN (_col4 is null) THEN (false) WHEN (_col8 is not null) THEN (true) WHEN (_col2 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (null) ELSE (false) END and CASE WHEN ((_col9 = 0L)) THEN (false) WHEN (_col12 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col10 < _col9)) THEN (null) ELSE (false) END) (type: boolean) + expressions: _col2 (type: int), (CASE WHEN ((_col4 = 0L)) THEN (false) WHEN (_col4 is null) THEN (false) WHEN (_col8 is not null) THEN (true) WHEN (_col2 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (null) ELSE (false) END and CASE WHEN ((_col11 = 0L)) THEN (false) WHEN (_col10 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col12 < _col11)) THEN (null) ELSE (false) END) (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 49 Data size: 392 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -3392,7 +3390,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[72][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[73][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product PREHOOK: query: SELECT p_size, p_size IN ( SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) AND p_name IN (SELECT min(p_name) from part) @@ -3407,32 +3405,32 @@ FROM part POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### -31 true +39 false 2 false 2 false -17 false -34 false -14 false -10 false -6 false -19 false -1 false -45 false -6 false 2 false -14 false -39 false -40 false -28 false +10 false +23 false 46 false -27 false 2 false -42 false +6 false 25 false -18 false +34 false +27 false +14 false +14 false 7 false +1 false +31 true +18 false +19 false +40 false +6 false +28 false 12 false -23 false +45 false +42 false +17 false PREHOOK: query: explain SELECT p_size, NOT EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) FROM part PREHOOK: type: QUERY @@ -3578,7 +3576,7 @@ POSTHOOK: Input: default@part 18 false 45 false Warning: Shuffle Join MERGEJOIN[50][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product -Warning: Shuffle Join MERGEJOIN[51][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select p_size, (p_size IN (select (select max(p_size) from part) as sb from part order by sb limit 1)) = true from part @@ -3600,13 +3598,13 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 5 <- Map 9 (CUSTOM_SIMPLE_EDGE), Reducer 4 (CUSTOM_SIMPLE_EDGE) Reducer 6 <- Reducer 5 (SIMPLE_EDGE) - Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) - Reducer 8 <- Reducer 5 (SIMPLE_EDGE) + Reducer 7 <- Reducer 5 (SIMPLE_EDGE) + Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -3619,9 +3617,10 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - sort order: + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) Select Operator expressions: p_size (type: int) outputColumnNames: p_size @@ -3654,31 +3653,29 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 26 Data size: 520 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 27 Data size: 116 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: 26 Data size: 520 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint), _col2 (type: bigint) + sort order: + Statistics: Num rows: 27 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: boolean) Reducer 3 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + 0 + 1 + outputColumnNames: _col0, _col2, _col3, _col4 Statistics: Num rows: 27 Data size: 548 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), CASE WHEN ((_col1 = 0L)) THEN (false) WHEN (_col4 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col2 < _col1)) THEN (null) ELSE (false) END (type: boolean) + expressions: _col0 (type: int), CASE WHEN ((_col3 = 0L)) THEN (false) WHEN (_col2 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (null) ELSE (false) END (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 27 Data size: 216 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -3719,12 +3716,32 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 Reduce Output Operator key expressions: _col0 (type: int) sort order: + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE - TopN Hash Memory Usage: 0.1 Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Select Operator @@ -3743,7 +3760,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 7 + Reducer 8 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3755,26 +3772,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 8 - Execution mode: vectorized, llap - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int) - outputColumnNames: _col0 - Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE - Limit - Number of rows: 1 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) Stage: Stage-0 Fetch Operator @@ -3783,7 +3780,7 @@ STAGE PLANS: ListSink Warning: Shuffle Join MERGEJOIN[50][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product -Warning: Shuffle Join MERGEJOIN[51][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select p_size, (p_size IN (select (select max(p_size) from part) as sb from part order by sb limit 1)) = true from part @@ -3796,32 +3793,32 @@ POSTHOOK: query: select p_size, (p_size IN POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### -1 false -2 false -2 false -2 false -2 false -6 false -6 false -7 false -10 false +17 false 12 false +46 true +1 false +31 false +45 false 14 false 14 false -17 false 18 false -19 false -23 false -25 false -27 false -28 false -31 false -34 false +10 false 39 false 40 false +7 false 42 false -45 false -46 true +6 false +6 false +34 false +2 false +2 false +2 false +2 false +28 false +27 false +25 false +23 false +19 false Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product @@ -4873,7 +4870,7 @@ POSTHOOK: Input: default@part 2 28 46 28 23 28 -Warning: Shuffle Join MERGEJOIN[92][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[102][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: explain select t1.p_size, (select count(*) from part p, part pp where p.p_size = pp.p_size and p.p_type = pp.p_type and (select sum(p_size) from part a1 where a1.p_partkey = p.p_partkey @@ -4899,11 +4896,12 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 11 <- Map 10 (SIMPLE_EDGE) - Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE) - Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE) - Reducer 5 <- Reducer 11 (ONE_TO_ONE_EDGE), Reducer 4 (SIMPLE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE) - Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) + Reducer 12 <- Map 11 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE) + Reducer 5 <- Map 10 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 12 (ONE_TO_ONE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) Reducer 9 <- Map 8 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: @@ -4923,6 +4921,26 @@ STAGE PLANS: Execution mode: vectorized, llap LLAP IO: no inputs Map 10 + Map Operator Tree: + TableScan + alias: pp + filterExpr: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int) + Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map 11 Map Operator Tree: TableScan alias: a1 @@ -4959,31 +4977,11 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: string), _col2 (type: int) - sort order: ++ - Map-reduce partition columns: _col1 (type: string), _col2 (type: int) + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) - Execution mode: vectorized, llap - LLAP IO: no inputs - Map 7 - Map Operator Tree: - TableScan - alias: pp - filterExpr: (p_size is not null and p_type is not null) (type: boolean) - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (p_size is not null and p_type is not null) (type: boolean) - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: p_type (type: string), p_size (type: int) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: int) - sort order: ++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: int) - Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs Map 8 @@ -5007,7 +5005,7 @@ STAGE PLANS: Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs - Reducer 11 + Reducer 12 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -5051,6 +5049,23 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Reducer 4 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 30 Data size: 3360 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string), _col2 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col2 (type: int) + Statistics: Num rows: 30 Data size: 3360 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int) + Reducer 5 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -5060,24 +5075,22 @@ STAGE PLANS: 0 _col1 (type: string), _col2 (type: int) 1 _col0 (type: string), _col1 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 28 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 32 Data size: 128 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: 28 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 5 + Statistics: Num rows: 32 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 - Inner Join 0 to 2 + Inner Join 0 to 1 keys: 0 _col0 (type: int) 1 _col0 (type: int) - 2 _col0 (type: int) - Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 32 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() mode: hash @@ -5087,7 +5100,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 6 + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -5132,7 +5145,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[92][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[102][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: select t1.p_size, (select count(*) from part p, part pp where p.p_size = pp.p_size and p.p_type = pp.p_type and (select sum(p_size) from part a1 where a1.p_partkey = p.p_partkey @@ -5175,7 +5188,7 @@ POSTHOOK: Input: default@part 2 28 46 28 23 28 -Warning: Shuffle Join MERGEJOIN[127][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[137][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain select t1.p_size, (select count(*) from part t2 where t2.p_partkey = t1.p_partkey group by t2.p_partkey), (select count(*) from part p, part pp where p.p_size = pp.p_size and p.p_type = pp.p_type @@ -5204,11 +5217,12 @@ STAGE PLANS: #### A masked pattern was here #### Edges: Reducer 10 <- Map 7 (SIMPLE_EDGE) - Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) + Reducer 11 <- Map 13 (SIMPLE_EDGE), Reducer 10 (ONE_TO_ONE_EDGE) + Reducer 12 <- Map 14 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE) Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE) Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE) Reducer 4 <- Map 1 (SIMPLE_EDGE) - Reducer 5 <- Reducer 10 (ONE_TO_ONE_EDGE), Reducer 12 (SIMPLE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE) + Reducer 5 <- Reducer 12 (SIMPLE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE) Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) Reducer 8 <- Map 7 (SIMPLE_EDGE) Reducer 9 <- Map 7 (SIMPLE_EDGE) @@ -5246,7 +5260,7 @@ STAGE PLANS: value expressions: _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs - Map 11 + Map 13 Map Operator Tree: TableScan alias: p @@ -5260,14 +5274,14 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: string), _col2 (type: int) - sort order: ++ - Map-reduce partition columns: _col1 (type: string), _col2 (type: int) + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int) + value expressions: _col1 (type: string), _col2 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs - Map 13 + Map 14 Map Operator Tree: TableScan alias: pp @@ -5356,6 +5370,23 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 4 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 11 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 30 Data size: 3360 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string), _col2 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col2 (type: int) + Statistics: Num rows: 30 Data size: 3360 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int) Reducer 12 Execution mode: llap Reduce Operator Tree: @@ -5366,12 +5397,12 @@ STAGE PLANS: 0 _col1 (type: string), _col2 (type: int) 1 _col0 (type: string), _col1 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 28 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 32 Data size: 128 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: 28 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 32 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE Reducer 2 Execution mode: llap Reduce Operator Tree: @@ -5441,13 +5472,11 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Left Outer Join 0 to 1 - Inner Join 0 to 2 + Inner Join 0 to 1 keys: 0 _col0 (type: int) 1 _col0 (type: int) - 2 _col0 (type: int) - Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 32 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() mode: hash @@ -5521,7 +5550,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[127][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[137][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product PREHOOK: query: select t1.p_size, (select count(*) from part t2 where t2.p_partkey = t1.p_partkey group by t2.p_partkey), (select count(*) from part p, part pp where p.p_size = pp.p_size and p.p_type = pp.p_type diff --git a/ql/src/test/results/clientpositive/llap/subquery_views.q.out b/ql/src/test/results/clientpositive/llap/subquery_views.q.out index a0bc64ebf1..4c723dce6b 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_views.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_views.q.out @@ -108,14 +108,15 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Map 9 (SIMPLE_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE), Reducer 6 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) - Reducer 4 <- Map 1 (SIMPLE_EDGE) + Reducer 11 <- Map 10 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (ONE_TO_ONE_EDGE), Reducer 8 (SIMPLE_EDGE) Reducer 5 <- Map 1 (SIMPLE_EDGE) - Reducer 6 <- Reducer 10 (ONE_TO_ONE_EDGE), Reducer 5 (SIMPLE_EDGE) - Reducer 7 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE) - Reducer 8 <- Map 1 (SIMPLE_EDGE) + Reducer 6 <- Map 1 (SIMPLE_EDGE) + Reducer 7 <- Reducer 11 (ONE_TO_ONE_EDGE), Reducer 6 (SIMPLE_EDGE) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE) + Reducer 9 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -195,7 +196,7 @@ STAGE PLANS: value expressions: _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized, llap LLAP IO: no inputs - Map 9 + Map 10 Map Operator Tree: TableScan alias: b @@ -218,7 +219,7 @@ STAGE PLANS: Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs - Reducer 10 + Reducer 11 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -242,40 +243,61 @@ STAGE PLANS: 0 _col0 (type: string), _col1 (type: string) 1 _col0 (type: string), _col1 (type: string) 2 _col3 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1, _col4, _col5, _col8 - Statistics: Num rows: 27 Data size: 5346 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col4, _col5, _col8 + Statistics: Num rows: 27 Data size: 2889 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: CASE WHEN ((_col4 = 0L)) THEN (true) WHEN (_col4 is null) THEN (true) WHEN (_col8 is not null) THEN (false) WHEN ((_col5 < _col4)) THEN (false) ELSE (true) END (type: boolean) - Statistics: Num rows: 13 Data size: 2574 Basic stats: COMPLETE Column stats: COMPLETE + predicate: CASE WHEN ((_col4 = 0L)) THEN (true) WHEN (_col4 is null) THEN (true) WHEN (_col8 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (false) ELSE (true) END (type: boolean) + Statistics: Num rows: 13 Data size: 1391 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string) + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1391 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 4 Execution mode: llap Reduce Operator Tree: Merge Join Operator condition map: - Left Semi Join 0 to 1 + Inner Join 0 to 1 keys: 0 _col0 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1 + outputColumnNames: _col1, _col2 Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE - 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 - Reducer 4 + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Reducer 5 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -290,7 +312,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 27 Data size: 5238 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: bigint), _col3 (type: bigint) - Reducer 5 + Reducer 6 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -308,7 +330,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 27 Data size: 4914 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string), _col2 (type: boolean) - Reducer 6 + Reducer 7 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -331,7 +353,7 @@ STAGE PLANS: Map-reduce partition columns: _col3 (type: string), _col1 (type: string) Statistics: Num rows: 27 Data size: 4914 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: boolean) - Reducer 7 + Reducer 8 Execution mode: llap Reduce Operator Tree: Merge Join Operator @@ -342,26 +364,22 @@ STAGE PLANS: 0 _col0 (type: string), _col1 (type: string) 1 _col0 (type: string), _col1 (type: string) 2 _col3 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col4, _col5, _col8 - Statistics: Num rows: 27 Data size: 2889 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col4, _col5, _col8 + Statistics: Num rows: 27 Data size: 5346 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: CASE WHEN ((_col4 = 0L)) THEN (true) WHEN (_col4 is null) THEN (true) WHEN (_col8 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (false) ELSE (true) END (type: boolean) - Statistics: Num rows: 13 Data size: 1391 Basic stats: COMPLETE Column stats: COMPLETE + predicate: CASE WHEN ((_col4 = 0L)) THEN (true) WHEN (_col4 is null) THEN (true) WHEN (_col8 is not null) THEN (false) WHEN ((_col5 < _col4)) THEN (false) ELSE (true) END (type: boolean) + Statistics: Num rows: 13 Data size: 2574 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 13 Data size: 1131 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 8 + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 2314 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Reducer 9 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -426,7 +444,7 @@ POSTHOOK: type: CREATEVIEW POSTHOOK: Input: default@src POSTHOOK: Output: database:default POSTHOOK: Output: default@cv3 -POSTHOOK: Lineage: cv3._c2 EXPRESSION [(src)b.null, ] +POSTHOOK: Lineage: cv3._c2 EXPRESSION [(src)src.null, (src)b.null, ] POSTHOOK: Lineage: cv3.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: cv3.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: describe extended cv3 diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out index a9180bcdc6..c20bc71257 100644 --- a/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out +++ b/ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out @@ -1,4 +1,4 @@ -Warning: Map Join MAPJOIN[32][bigTable=?] in task 'Map 1' is a cross product +Warning: Map Join MAPJOIN[34][bigTable=?] in task 'Map 1' is a cross product PREHOOK: query: explain vectorization expression select * from src @@ -32,8 +32,8 @@ STAGE PLANS: Edges: Map 1 <- Reducer 4 (BROADCAST_EDGE), Reducer 5 (BROADCAST_EDGE) Reducer 2 <- Map 1 (SIMPLE_EDGE) - Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE) - Reducer 5 <- Map 3 (SIMPLE_EDGE) + Reducer 4 <- Map 3 (SIMPLE_EDGE) + Reducer 5 <- Map 3 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -53,58 +53,66 @@ STAGE PLANS: Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Map Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 + 0 _col0 (type: string) + 1 _col0 (type: string) Map Join Vectorization: - className: VectorMapJoinInnerMultiKeyOperator + className: VectorMapJoinOuterStringOperator native: true - nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true + nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Outer Join has keys IS true, Optimized Table and Supports Key Types IS true hashTableImplementationType: OPTIMIZED - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col3 input vertices: 1 Reducer 4 - Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 895 Data size: 160894 Basic stats: COMPLETE Column stats: COMPLETE Map Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: string) - 1 _col0 (type: string) + 0 + 1 Map Join Vectorization: - className: VectorMapJoinOuterStringOperator + className: VectorMapJoinInnerMultiKeyOperator native: true - nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Outer Join has keys IS true, Optimized Table and Supports Key Types IS true + nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true hashTableImplementationType: OPTIMIZED - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 input vertices: 1 Reducer 5 Statistics: Num rows: 895 Data size: 175214 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - Filter Vectorization: - className: VectorFilterOperator + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Select Vectorization: + className: VectorSelectOperator native: true - predicateExpression: FilterExprOrExpr(children: FilterLongColEqualLongScalar(col 3:bigint, val 0), FilterExprAndExpr(children: SelectColumnIsNull(col 5:boolean), SelectColumnIsNotNull(col 0:string), FilterLongColGreaterEqualLongColumn(col 4:bigint, col 3:bigint))) - predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + projectedOutputColumnNums: [0, 1, 4, 5, 3] Statistics: Num rows: 895 Data size: 175214 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Select Vectorization: - className: VectorSelectOperator + Filter Operator + Filter Vectorization: + className: VectorFilterOperator native: true - projectedOutputColumnNums: [0, 1] - Statistics: Num rows: 895 Data size: 159310 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + predicateExpression: FilterExprOrExpr(children: FilterLongColEqualLongScalar(col 4:bigint, val 0), FilterExprAndExpr(children: SelectColumnIsNull(col 3:boolean), SelectColumnIsNotNull(col 0:string), FilterLongColGreaterEqualLongColumn(col 5:bigint, col 4:bigint))) + predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 895 Data size: 175214 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Select Vectorization: + className: VectorSelectOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + projectedOutputColumnNums: [0, 1] Statistics: Num rows: 895 Data size: 159310 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string) + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + Statistics: Num rows: 895 Data size: 159310 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) Execution mode: vectorized, llap LLAP IO: no inputs Map Vectorization: @@ -123,6 +131,27 @@ STAGE PLANS: Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE TableScan Vectorization: native: true + Group By Operator + Group By Vectorization: + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 0:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [] + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: key (type: string) outputColumnNames: key @@ -151,27 +180,6 @@ STAGE PLANS: nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Group By Operator - Group By Vectorization: - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 0:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [] - keys: key (type: string) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Map Vectorization: @@ -211,35 +219,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Reducer 4 - Execution mode: vectorized, llap - Reduce Vectorization: - enabled: true - enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true - allNative: false - usesVectorUDFAdaptor: false - vectorized: true - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - Group By Vectorization: - aggregators: VectorUDAFCountMerge(col 0:bigint) -> bigint, VectorUDAFCountMerge(col 1:bigint) -> bigint - className: VectorGroupByOperator - groupByMode: MERGEPARTIAL - native: false - vectorProcessingMode: GLOBAL - projectedOutputColumnNums: [0, 1] - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Reduce Sink Vectorization: - className: VectorReduceSinkEmptyKeyOperator - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 5 Execution mode: vectorized, llap Reduce Vectorization: enabled: true @@ -279,6 +258,35 @@ STAGE PLANS: nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 5 + Execution mode: vectorized, llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + allNative: false + usesVectorUDFAdaptor: false + vectorized: true + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + Group By Vectorization: + aggregators: VectorUDAFCountMerge(col 0:bigint) -> bigint, VectorUDAFCountMerge(col 1:bigint) -> bigint + className: VectorGroupByOperator + groupByMode: MERGEPARTIAL + native: false + vectorProcessingMode: GLOBAL + projectedOutputColumnNums: [0, 1] + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Reduce Sink Vectorization: + className: VectorReduceSinkEmptyKeyOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Stage: Stage-0 Fetch Operator @@ -286,7 +294,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Map Join MAPJOIN[32][bigTable=?] in task 'Map 1' is a cross product +Warning: Map Join MAPJOIN[34][bigTable=?] in task 'Map 1' is a cross product PREHOOK: query: select * from src where not key in @@ -315,7 +323,7 @@ POSTHOOK: Output: database:default POSTHOOK: Output: default@orcsrc POSTHOOK: Lineage: orcsrc.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: orcsrc.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] -Warning: Map Join MAPJOIN[32][bigTable=?] in task 'Map 1' is a cross product +Warning: Map Join MAPJOIN[34][bigTable=?] in task 'Map 1' is a cross product PREHOOK: query: select * from orcsrc where not key in @@ -332,7 +340,7 @@ order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@orcsrc #### A masked pattern was here #### -Warning: Map Join MAPJOIN[32][bigTable=?] in task 'Map 1' is a cross product +Warning: Map Join MAPJOIN[34][bigTable=?] in task 'Map 1' is a cross product PREHOOK: query: select * from orcsrc where not key in diff --git a/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out b/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out index 3b8ee96a40..ad57fcd666 100644 --- a/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out +++ b/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out @@ -27,45 +27,43 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE) + Reducer 3 <- Map 1 (BROADCAST_EDGE), Map 2 (SIMPLE_EDGE), Reducer 5 (BROADCAST_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan - alias: lineitem - filterExpr: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + alias: li + filterExpr: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean) + Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE TableScan Vectorization: native: true Filter Operator Filter Vectorization: className: VectorFilterOperator native: true - predicateExpression: SelectColumnIsNotNull(col 1:int) - predicate: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - Group By Vectorization: - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:int - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [] - keys: l_partkey (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + predicateExpression: FilterExprAndExpr(children: FilterLongColEqualLongScalar(col 3:int, val 1), SelectColumnIsNotNull(col 1:int), SelectColumnIsNotNull(col 0:int)) + predicate: ((l_linenumber = 1) and l_orderkey is not null and l_partkey is not null) (type: boolean) + Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int) + outputColumnNames: _col0, _col1, _col2 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2] + Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int) + key expressions: _col1 (type: int) sort order: + - Map-reduce partition columns: _col0 (type: int) + Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: int) Execution mode: vectorized, llap LLAP IO: no inputs Map Vectorization: @@ -74,42 +72,45 @@ STAGE PLANS: inputFormatFeatureSupport: [DECIMAL_64] featureSupportInUse: [DECIMAL_64] inputFileFormats: org.apache.hadoop.mapred.TextInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true - Map 3 + Map 2 Map Operator Tree: TableScan - alias: li - filterExpr: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean) - Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE + alias: lineitem + filterExpr: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE TableScan Vectorization: native: true Filter Operator Filter Vectorization: className: VectorFilterOperator native: true - predicateExpression: FilterExprAndExpr(children: FilterLongColEqualLongScalar(col 3:int, val 1), SelectColumnIsNotNull(col 1:int), SelectColumnIsNotNull(col 0:int)) - predicate: ((l_linenumber = 1) and l_orderkey is not null and l_partkey is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int) - outputColumnNames: _col0, _col1, _col2 - Select Vectorization: - className: VectorSelectOperator - native: true - projectedOutputColumnNums: [0, 1, 2] - Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + predicateExpression: SelectColumnIsNotNull(col 1:int) + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + Group By Vectorization: + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:int + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [] + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: int) + key expressions: _col0 (type: int) sort order: + - Map-reduce partition columns: _col1 (type: int) + Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col2 (type: int) + Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Map Vectorization: @@ -118,7 +119,7 @@ STAGE PLANS: inputFormatFeatureSupport: [DECIMAL_64] featureSupportInUse: [DECIMAL_64] inputFileFormats: org.apache.hadoop.mapred.TextInputFormat - allNative: true + allNative: false usesVectorUDFAdaptor: false vectorized: true Map 4 @@ -138,12 +139,12 @@ STAGE PLANS: Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: l_orderkey (type: int) - outputColumnNames: _col0 + outputColumnNames: l_orderkey Select Vectorization: className: VectorSelectOperator native: true projectedOutputColumnNums: [0] - Statistics: Num rows: 14 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator Group By Vectorization: className: VectorGroupByOperator @@ -152,7 +153,7 @@ STAGE PLANS: native: false vectorProcessingMode: HASH projectedOutputColumnNums: [] - keys: _col0 (type: int) + keys: l_orderkey (type: int) mode: hash outputColumnNames: _col0 Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE @@ -176,7 +177,7 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true - Reducer 2 + Reducer 3 Execution mode: vectorized, llap Reduce Vectorization: enabled: true @@ -201,34 +202,34 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col1 (type: int) + 0 _col1 (type: int) + 1 _col0 (type: int) Map Join Vectorization: className: VectorMapJoinInnerLongOperator native: true nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true hashTableImplementationType: OPTIMIZED - outputColumnNames: _col0, _col1, _col3 + outputColumnNames: _col0, _col1, _col2 input vertices: - 1 Map 3 + 0 Map 1 Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE Map Join Operator condition map: - Left Semi Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col1 (type: int) + 0 _col0 (type: int) 1 _col0 (type: int) Map Join Vectorization: - className: VectorMapJoinLeftSemiLongOperator + className: VectorMapJoinInnerBigOnlyLongOperator native: true nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true hashTableImplementationType: OPTIMIZED - outputColumnNames: _col0, _col3 + outputColumnNames: _col1, _col2 input vertices: - 1 Map 4 + 1 Reducer 5 Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col3 (type: int) + expressions: _col1 (type: int), _col2 (type: int) outputColumnNames: _col0, _col1 Select Vectorization: className: VectorSelectOperator @@ -245,6 +246,36 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + allNative: false + usesVectorUDFAdaptor: false + vectorized: true + Reduce Operator Tree: + Group By Operator + Group By Vectorization: + className: VectorGroupByOperator + groupByMode: MERGEPARTIAL + keyExpressions: col 0:int + native: false + vectorProcessingMode: MERGE_PARTIAL + projectedOutputColumnNums: [] + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Reduce Sink Vectorization: + className: VectorReduceSinkLongOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Stage: Stage-0 Fetch Operator @@ -305,45 +336,90 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE) + Map 1 <- Map 2 (BROADCAST_EDGE), Map 3 (BROADCAST_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan - alias: lineitem - filterExpr: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + alias: li + filterExpr: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean) + Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE TableScan Vectorization: native: true Filter Operator Filter Vectorization: className: VectorFilterOperator native: true - predicateExpression: SelectColumnIsNotNull(col 1:int) - predicate: l_partkey is not null (type: boolean) - Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - Group By Vectorization: - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:int - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [] - keys: l_partkey (type: int) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int) - sort order: + - Map-reduce partition columns: _col0 (type: int) - Reduce Sink Vectorization: - className: VectorReduceSinkLongOperator + predicateExpression: FilterExprAndExpr(children: FilterLongColEqualLongScalar(col 3:int, val 1), SelectColumnIsNotNull(col 1:int), SelectColumnIsNotNull(col 0:int)) + predicate: ((l_linenumber = 1) and l_orderkey is not null and l_partkey is not null) (type: boolean) + Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 17] + selectExpressions: ConstantVectorExpression(val 1) -> 17:int + Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + Map Join Vectorization: + bigTableValueExpressions: ConstantVectorExpression(val 1) -> 18:int + className: VectorMapJoinLeftSemiLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true + hashTableImplementationType: OPTIMIZED + outputColumnNames: _col0, _col1, _col2, _col3 + input vertices: + 1 Map 2 + Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col0 (type: int), _col2 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [1, 0, 2, 18] + Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: int), _col4 (type: int) + 1 _col0 (type: int), _col1 (type: int) + Map Join Vectorization: + className: VectorMapJoinLeftSemiMultiKeyOperator + native: true + nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true + hashTableImplementationType: OPTIMIZED + outputColumnNames: _col0, _col3 + input vertices: + 1 Map 3 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [1, 2] + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + File Sink Vectorization: + className: VectorFileSinkOperator + native: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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 Execution mode: vectorized, llap LLAP IO: no inputs Map Vectorization: @@ -355,40 +431,50 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true - Map 3 + Map 2 Map Operator Tree: TableScan - alias: li - filterExpr: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean) - Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE + alias: lineitem + filterExpr: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE TableScan Vectorization: native: true Filter Operator Filter Vectorization: className: VectorFilterOperator native: true - predicateExpression: FilterExprAndExpr(children: FilterLongColEqualLongScalar(col 3:int, val 1), SelectColumnIsNotNull(col 1:int), SelectColumnIsNotNull(col 0:int)) - predicate: ((l_linenumber = 1) and l_orderkey is not null and l_partkey is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + predicateExpression: SelectColumnIsNotNull(col 1:int) + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3 + expressions: l_partkey (type: int) + outputColumnNames: _col0 Select Vectorization: className: VectorSelectOperator native: true - projectedOutputColumnNums: [0, 1, 2, 17] - selectExpressions: ConstantVectorExpression(val 1) -> 17:int - Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col1 (type: int) - sort order: + - Map-reduce partition columns: _col1 (type: int) - Reduce Sink Vectorization: - className: VectorReduceSinkLongOperator - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int) + projectedOutputColumnNums: [1] + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + Group By Vectorization: + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:int + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [] + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Reduce Sink Vectorization: + className: VectorReduceSinkLongOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs Map Vectorization: @@ -397,10 +483,10 @@ STAGE PLANS: inputFormatFeatureSupport: [DECIMAL_64] featureSupportInUse: [DECIMAL_64] inputFileFormats: org.apache.hadoop.mapred.TextInputFormat - allNative: true + allNative: false usesVectorUDFAdaptor: false vectorized: true - Map 4 + Map 3 Map Operator Tree: TableScan alias: lineitem @@ -456,75 +542,6 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true - Reducer 2 - Execution mode: vectorized, llap - Reduce Vectorization: - enabled: true - enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true - allNative: false - usesVectorUDFAdaptor: false - vectorized: true - Reduce Operator Tree: - Group By Operator - Group By Vectorization: - className: VectorGroupByOperator - groupByMode: MERGEPARTIAL - keyExpressions: col 0:int - native: false - vectorProcessingMode: MERGE_PARTIAL - projectedOutputColumnNums: [] - keys: KEY._col0 (type: int) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Map Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col1 (type: int) - Map Join Vectorization: - className: VectorMapJoinInnerLongOperator - native: true - nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true - hashTableImplementationType: OPTIMIZED - outputColumnNames: _col0, _col1, _col3, _col4 - input vertices: - 1 Map 3 - Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE - Map Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col1 (type: int), _col4 (type: int) - 1 _col0 (type: int), _col1 (type: int) - Map Join Vectorization: - className: VectorMapJoinLeftSemiMultiKeyOperator - native: true - nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true - hashTableImplementationType: OPTIMIZED - outputColumnNames: _col0, _col3 - input vertices: - 1 Map 4 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col3 (type: int) - outputColumnNames: _col0, _col1 - Select Vectorization: - className: VectorSelectOperator - native: true - projectedOutputColumnNums: [0, 2] - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - File Sink Vectorization: - className: VectorFileSinkOperator - native: false - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - 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 diff --git a/ql/src/test/results/clientpositive/semijoin5.q.out b/ql/src/test/results/clientpositive/semijoin5.q.out index c047c6d265..7bf7276ee9 100644 --- a/ql/src/test/results/clientpositive/semijoin5.q.out +++ b/ql/src/test/results/clientpositive/semijoin5.q.out @@ -197,38 +197,38 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan - alias: tt1 - filterExpr: decimal2612_col_77 is not null (type: boolean) + alias: tt2 + filterExpr: (timestamp_col_18 is not null and decimal1911_col_16 is not null) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: decimal2612_col_77 is not null (type: boolean) + predicate: (decimal1911_col_16 is not null and timestamp_col_18 is not null) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: decimal2612_col_77 (type: decimal(26,12)) - outputColumnNames: _col0 + expressions: decimal1911_col_16 (type: decimal(19,11)), timestamp_col_18 (type: timestamp) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator key expressions: _col0 (type: decimal(26,12)) sort order: + Map-reduce partition columns: _col0 (type: decimal(26,12)) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: timestamp) TableScan - alias: tt2 - filterExpr: (timestamp_col_18 is not null and decimal1911_col_16 is not null) (type: boolean) + alias: tt1 + filterExpr: decimal2612_col_77 is not null (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: (decimal1911_col_16 is not null and timestamp_col_18 is not null) (type: boolean) + predicate: decimal2612_col_77 is not null (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: decimal1911_col_16 (type: decimal(19,11)), timestamp_col_18 (type: timestamp) - outputColumnNames: _col0, _col1 + expressions: decimal2612_col_77 (type: decimal(26,12)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator key expressions: _col0 (type: decimal(26,12)) sort order: + Map-reduce partition columns: _col0 (type: decimal(26,12)) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col1 (type: timestamp) Reduce Operator Tree: Join Operator condition map: @@ -236,10 +236,10 @@ STAGE PLANS: keys: 0 _col0 (type: decimal(26,12)) 1 _col0 (type: decimal(26,12)) - outputColumnNames: _col2 + outputColumnNames: _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col2 (type: timestamp), -92 (type: int) + expressions: _col1 (type: timestamp), -92 (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Group By Operator diff --git a/ql/src/test/results/clientpositive/subquery_notin_having.q.out b/ql/src/test/results/clientpositive/subquery_notin_having.q.out index dd31d4ab6a..30580ab549 100644 --- a/ql/src/test/results/clientpositive/subquery_notin_having.q.out +++ b/ql/src/test/results/clientpositive/subquery_notin_having.q.out @@ -1,4 +1,4 @@ -Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product PREHOOK: query: explain select key, count(*) from src @@ -72,23 +72,27 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - sort order: + 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: _col0 (type: string), _col1 (type: bigint) + value expressions: _col1 (type: bigint) TableScan Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) Reduce Operator Tree: Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 250 Data size: 6906 Basic stats: COMPLETE Column stats: NONE + 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 File Output Operator compressed: false table: @@ -101,76 +105,43 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 250 Data size: 6906 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint) + sort order: + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: bigint), _col3 (type: boolean) TableScan Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: boolean) + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reduce Operator Tree: Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + 0 + 1 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 Statistics: Num rows: 275 Data size: 7596 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) - Statistics: Num rows: 182 Data size: 5027 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: bigint) - outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Statistics: Num rows: 275 Data size: 7596 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) Statistics: Num rows: 182 Data size: 5027 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 Statistics: Num rows: 182 Data size: 5027 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 + File Output Operator + compressed: false + Statistics: Num rows: 182 Data size: 5027 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-4 - Map Reduce - Map Operator Tree: - TableScan - alias: s1 - filterExpr: (key > '12') (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: (key > '12') (type: boolean) - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE - Group By Operator - aggregations: count(), count(key) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 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-5 Map Reduce Map Operator Tree: TableScan @@ -208,6 +179,39 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: s1 + filterExpr: (key > '12') (type: boolean) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '12') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(key) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 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 @@ -508,7 +512,7 @@ POSTHOOK: Input: default@part #### A masked pattern was here #### Manufacturer#1 1173.15 Manufacturer#2 1690.68 -Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product PREHOOK: query: explain select b.p_mfgr, min(p_retailprice) from part b @@ -537,11 +541,11 @@ POSTHOOK: Input: default@part #### A masked pattern was here #### STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-2 depends on stages: Stage-1, Stage-5 + Stage-2 depends on stages: Stage-1, Stage-4 Stage-3 depends on stages: Stage-2, Stage-6 Stage-4 is a root stage - Stage-5 depends on stages: Stage-4 - Stage-6 is a root stage + Stage-5 is a root stage + Stage-6 depends on stages: Stage-5 Stage-0 depends on stages: Stage-3 STAGE PLANS: @@ -587,23 +591,27 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - sort order: + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: string), _col1 (type: double) + value expressions: _col1 (type: double) TableScan Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) Reduce Operator Tree: Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 13 Data size: 1898 Basic stats: COMPLETE Column stats: NONE + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false table: @@ -616,43 +624,87 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 13 Data size: 1898 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: double), _col2 (type: bigint), _col3 (type: bigint) + sort order: + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: double), _col3 (type: boolean) TableScan Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: boolean) + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reduce Operator Tree: Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 - Statistics: Num rows: 14 Data size: 2087 Basic stats: COMPLETE Column stats: NONE + 0 + 1 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 + Statistics: Num rows: 14 Data size: 2080 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: double), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Statistics: Num rows: 14 Data size: 2080 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 9 Data size: 1337 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 1337 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 9 Data size: 1337 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-4 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_mfgr (type: string), p_retailprice (type: double) + outputColumnNames: p_mfgr, p_retailprice + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_retailprice), min(p_retailprice) + keys: p_mfgr (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: double), _col2 (type: double) + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0), min(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) - Statistics: Num rows: 9 Data size: 1341 Basic stats: COMPLETE Column stats: NONE + predicate: ((_col1 - _col2) > 600.0D) (type: boolean) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: string), _col1 (type: double) + expressions: _col0 (type: string), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 9 Data size: 1341 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 9 Data size: 1341 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 + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-4 + Stage: Stage-5 Map Reduce Map Operator Tree: TableScan @@ -701,7 +753,7 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-5 + Stage: Stage-6 Map Reduce Map Operator Tree: TableScan @@ -723,57 +775,13 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-6 - Map Reduce - Map Operator Tree: - TableScan - alias: a - Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: p_mfgr (type: string), p_retailprice (type: double) - outputColumnNames: p_mfgr, p_retailprice - Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE - Group By Operator - aggregations: max(p_retailprice), min(p_retailprice) - keys: p_mfgr (type: string) - mode: hash - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: double), _col2 (type: double) - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: max(VALUE._col0), min(VALUE._col1) - keys: KEY._col0 (type: string) - mode: mergepartial - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col1 - _col2) > 600.0D) (type: boolean) - Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 4 Data size: 484 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 Processor Tree: ListSink -Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product PREHOOK: query: select b.p_mfgr, min(p_retailprice) from part b group by b.p_mfgr @@ -798,8 +806,8 @@ having b.p_mfgr not in POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### -Manufacturer#1 1173.15 Manufacturer#2 1690.68 +Manufacturer#1 1173.15 PREHOOK: query: CREATE TABLE t1_n145 (c1 INT, c2 CHAR(100)) PREHOOK: type: CREATETABLE PREHOOK: Output: database:default @@ -835,7 +843,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@t2_n85 POSTHOOK: Lineage: t2_n85.c1 SCRIPT [] -Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product PREHOOK: query: explain SELECT c1 FROM t1_n145 group by c1 having c1 NOT IN (SELECT c1 FROM t2_n85) PREHOOK: type: QUERY PREHOOK: Input: default@t1_n145 @@ -894,23 +902,26 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - sort order: + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 156 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int) TableScan Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) Reduce Operator Tree: Join Operator condition map: - Inner Join 0 to 1 + Left Outer Join 0 to 1 keys: - 0 - 1 - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: NONE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 2 Data size: 171 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false table: @@ -923,76 +934,43 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - key expressions: _col0 (type: int) - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: bigint), _col2 (type: bigint) + sort order: + Statistics: Num rows: 2 Data size: 171 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: boolean) TableScan Reduce Output Operator - key expressions: _col0 (type: int) - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: boolean) + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Reduce Operator Tree: Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 - Statistics: Num rows: 2 Data size: 209 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) - Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 104 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 + 0 + 1 + outputColumnNames: _col0, _col2, _col3, _col4 + Statistics: Num rows: 2 Data size: 205 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col3 (type: bigint), _col4 (type: bigint), _col2 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 2 Data size: 205 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 = 0L) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 102 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-4 - Map Reduce - Map Operator Tree: - TableScan - alias: t2_n85 - Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: c1 (type: int) - outputColumnNames: c1 - Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE - Group By Operator - aggregations: count(), count(c1) - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 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-5 Map Reduce Map Operator Tree: TableScan @@ -1030,13 +1008,46 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: t2_n85 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: c1 (type: int) + outputColumnNames: c1 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(c1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 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 Processor Tree: ListSink -Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-3:MAPRED' is a cross product PREHOOK: query: SELECT c1 FROM t1_n145 group by c1 having c1 NOT IN (SELECT c1 FROM t2_n85) PREHOOK: type: QUERY PREHOOK: Input: default@t1_n145 diff --git a/ql/src/test/results/clientpositive/vector_groupby_mapjoin.q.out b/ql/src/test/results/clientpositive/vector_groupby_mapjoin.q.out index 70cdd7ae5b..1f60366c5e 100644 --- a/ql/src/test/results/clientpositive/vector_groupby_mapjoin.q.out +++ b/ql/src/test/results/clientpositive/vector_groupby_mapjoin.q.out @@ -1,4 +1,6 @@ -Warning: Map Join MAPJOIN[42][bigTable=?] in task 'Stage-8:MAPRED' is a cross product +Warning: Map Join MAPJOIN[53][bigTable=?] in task 'Stage-7:MAPRED' is a cross product +Warning: Map Join MAPJOIN[43][bigTable=?] in task 'Stage-6:MAPRED' is a cross product +Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product PREHOOK: query: explain vectorization expression select * from src @@ -23,13 +25,17 @@ PLAN VECTORIZATION: STAGE DEPENDENCIES: Stage-4 is a root stage - Stage-10 depends on stages: Stage-4 - Stage-8 depends on stages: Stage-10 - Stage-7 depends on stages: Stage-5, Stage-8 , consists of Stage-9, Stage-2 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 - Stage-3 depends on stages: Stage-2, Stage-6 + Stage-10 depends on stages: Stage-4 , consists of Stage-13, Stage-1 + Stage-13 has a backup stage: Stage-1 + Stage-9 depends on stages: Stage-13 + Stage-8 depends on stages: Stage-1, Stage-5, Stage-9 , consists of Stage-11, Stage-12, Stage-2 + Stage-11 has a backup stage: Stage-2 + Stage-6 depends on stages: Stage-11 + Stage-3 depends on stages: Stage-2, Stage-6, Stage-7 + Stage-12 has a backup stage: Stage-2 + Stage-7 depends on stages: Stage-12 Stage-2 + Stage-1 Stage-5 is a root stage Stage-0 depends on stages: Stage-3 @@ -51,26 +57,27 @@ STAGE PLANS: projectedOutputColumnNums: [0] Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(), count(key) Group By Vectorization: - aggregators: VectorUDAFCountStar(*) -> bigint, VectorUDAFCount(col 0:string) -> bigint className: VectorGroupByOperator groupByMode: HASH + keyExpressions: col 0:string native: false vectorProcessingMode: HASH - projectedOutputColumnNums: [0, 1] + projectedOutputColumnNums: [] + keys: key (type: string) mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - sort order: + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint), _col1 (type: bigint) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: enabled: true @@ -87,95 +94,107 @@ STAGE PLANS: enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Reduce Operator Tree: Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 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 + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 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-10 + Conditional Operator + + Stage: Stage-13 Map Reduce Local Work Alias -> Map Local Tables: - $hdt$_0:src + $hdt$_0:$INTNAME Fetch Operator limit: -1 Alias -> Map Local Operator Tree: - $hdt$_0:src + $hdt$_0:$INTNAME + TableScan + HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + + Stage: Stage-9 + Map Reduce + Map Operator Tree: TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true Select Operator expressions: key (type: string), value (type: string) outputColumnNames: _col0, _col1 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - HashTable Sink Operator + Map Join Operator + condition map: + Left Outer Join 0 to 1 keys: - 0 - 1 - - Stage: Stage-8 - Map Reduce - Map Operator Tree: - TableScan - TableScan Vectorization: - native: true - Map Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 - 1 - Map Join Vectorization: - bigTableValueExpressions: col 0:bigint, col 1:bigint - className: VectorMapJoinOperator - native: false - nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 500 Data size: 13812 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - File Sink Vectorization: - className: VectorFileSinkOperator + 0 _col0 (type: string) + 1 _col0 (type: string) + Map Join Vectorization: + bigTableKeyExpressions: col 0:string + bigTableValueExpressions: col 0:string, col 1:string + className: VectorMapJoinOperator native: 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 + nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Outer Join has keys IS true, Optimized Table and Supports Key Types IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + File Sink Vectorization: + className: VectorFileSinkOperator + native: 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 Execution mode: vectorized Map Vectorization: enabled: true enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true - inputFormatFeatureSupport: [] - featureSupportInUse: [] - inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat + inputFormatFeatureSupport: [DECIMAL_64] + featureSupportInUse: [DECIMAL_64] + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat allNative: false usesVectorUDFAdaptor: false vectorized: true Local Work: Map Reduce Local Work - Stage: Stage-7 + Stage: Stage-8 Conditional Operator - Stage: Stage-9 + Stage: Stage-11 Map Reduce Local Work Alias -> Map Local Tables: - $INTNAME1 + $hdt$_0:$INTNAME1 Fetch Operator limit: -1 Alias -> Map Local Operator Tree: - $INTNAME1 + $hdt$_0:$INTNAME1 TableScan HashTable Sink Operator keys: - 0 _col0 (type: string) - 1 _col0 (type: string) + 0 + 1 Stage: Stage-6 Map Reduce @@ -185,43 +204,50 @@ STAGE PLANS: native: true Map Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: string) - 1 _col0 (type: string) + 0 + 1 Map Join Vectorization: - bigTableKeyExpressions: col 0:string - bigTableValueExpressions: col 0:string, col 1:string, col 2:bigint, col 3:bigint + bigTableValueExpressions: col 0:string, col 1:string, col 2:boolean className: VectorMapJoinOperator native: false - nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Outer Join has keys IS true, Optimized Table and Supports Key Types IS true + nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 Statistics: Num rows: 550 Data size: 15193 Basic stats: COMPLETE Column stats: NONE - Filter Operator - Filter Vectorization: - className: VectorFilterOperator + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Select Vectorization: + className: VectorSelectOperator native: true - predicateExpression: FilterExprOrExpr(children: FilterLongColEqualLongScalar(col 2:bigint, val 0), FilterExprAndExpr(children: SelectColumnIsNull(col 4:boolean), SelectColumnIsNotNull(col 0:string), FilterLongColGreaterEqualLongColumn(col 3:bigint, col 2:bigint))) - predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) - Statistics: Num rows: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Select Vectorization: - className: VectorSelectOperator + projectedOutputColumnNums: [0, 1, 3, 4, 2] + Statistics: Num rows: 550 Data size: 15193 Basic stats: COMPLETE Column stats: NONE + Filter Operator + Filter Vectorization: + className: VectorFilterOperator native: true - projectedOutputColumnNums: [0, 1] + predicateExpression: FilterExprOrExpr(children: FilterLongColEqualLongScalar(col 3:bigint, val 0), FilterExprAndExpr(children: SelectColumnIsNull(col 2:boolean), SelectColumnIsNotNull(col 0:string), FilterLongColGreaterEqualLongColumn(col 4:bigint, col 3:bigint))) + predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) Statistics: Num rows: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - File Sink Vectorization: - className: VectorFileSinkOperator - native: 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 + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Statistics: Num rows: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + File Sink Vectorization: + className: VectorFileSinkOperator + native: 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 Execution mode: vectorized Map Vectorization: enabled: true @@ -278,16 +304,141 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Stage: Stage-12 + Map Reduce Local Work + Alias -> Map Local Tables: + $hdt$_0:$INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $hdt$_0:$INTNAME + TableScan + HashTable Sink Operator + keys: + 0 + 1 + + Stage: Stage-7 + Map Reduce + Map Operator Tree: + TableScan + TableScan Vectorization: + native: true + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + Map Join Vectorization: + bigTableValueExpressions: col 0:bigint, col 1:bigint + className: VectorMapJoinOperator + native: false + nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false + outputColumnNames: _col0, _col1, _col3, _col4, _col5 + Statistics: Num rows: 550 Data size: 15193 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 3, 4, 2] + Statistics: Num rows: 550 Data size: 15193 Basic stats: COMPLETE Column stats: NONE + Filter Operator + Filter Vectorization: + className: VectorFilterOperator + native: true + predicateExpression: FilterExprOrExpr(children: FilterLongColEqualLongScalar(col 3:bigint, val 0), FilterExprAndExpr(children: SelectColumnIsNull(col 2:boolean), SelectColumnIsNotNull(col 0:string), FilterLongColGreaterEqualLongColumn(col 4:bigint, col 3:bigint))) + predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Statistics: Num rows: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + File Sink Vectorization: + className: VectorFileSinkOperator + native: 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 + Execution mode: vectorized + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + inputFormatFeatureSupport: [] + featureSupportInUse: [] + inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat + allNative: false + usesVectorUDFAdaptor: false + vectorized: true + Local Work: + Map Reduce Local Work + Stage: Stage-2 Map Reduce Map Operator Tree: TableScan Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 500 Data size: 13812 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string), _col2 (type: bigint), _col3 (type: bigint) + sort order: + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: boolean) + TableScan + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col3, _col4, _col5 + Statistics: Num rows: 550 Data size: 15193 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint), _col3 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Statistics: Num rows: 550 Data size: 15193 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 366 Data size: 10110 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-1 + Map Reduce + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) TableScan Reduce Output Operator key expressions: _col0 (type: string) @@ -302,21 +453,14 @@ STAGE PLANS: keys: 0 _col0 (type: string) 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 - Statistics: Num rows: 550 Data size: 15193 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((_col2 = 0L) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) - Statistics: Num rows: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 366 Data size: 10110 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 + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 550 Data size: 5843 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-5 Map Reduce @@ -335,27 +479,26 @@ STAGE PLANS: projectedOutputColumnNums: [0] Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Group By Operator + aggregations: count(), count(key) Group By Vectorization: + aggregators: VectorUDAFCountStar(*) -> bigint, VectorUDAFCount(col 0:string) -> bigint className: VectorGroupByOperator groupByMode: HASH - keyExpressions: col 0:string native: false vectorProcessingMode: HASH - projectedOutputColumnNums: [] - keys: key (type: string) + projectedOutputColumnNums: [0, 1] mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + sort order: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized Map Vectorization: enabled: true @@ -372,20 +515,16 @@ STAGE PLANS: enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Reduce Operator Tree: Group By Operator - keys: KEY._col0 (type: string) + aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), true (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 250 Data size: 2656 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 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 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 @@ -393,7 +532,9 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Map Join MAPJOIN[42][bigTable=?] in task 'Stage-8:MAPRED' is a cross product +Warning: Map Join MAPJOIN[53][bigTable=?] in task 'Stage-7:MAPRED' is a cross product +Warning: Map Join MAPJOIN[43][bigTable=?] in task 'Stage-6:MAPRED' is a cross product +Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product PREHOOK: query: select * from src where not key in @@ -422,7 +563,9 @@ POSTHOOK: Output: database:default POSTHOOK: Output: default@orcsrc POSTHOOK: Lineage: orcsrc.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: orcsrc.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] -Warning: Map Join MAPJOIN[42][bigTable=?] in task 'Stage-8:MAPRED' is a cross product +Warning: Map Join MAPJOIN[53][bigTable=?] in task 'Stage-7:MAPRED' is a cross product +Warning: Map Join MAPJOIN[43][bigTable=?] in task 'Stage-6:MAPRED' is a cross product +Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product PREHOOK: query: select * from orcsrc where not key in @@ -439,7 +582,9 @@ order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@orcsrc #### A masked pattern was here #### -Warning: Map Join MAPJOIN[42][bigTable=?] in task 'Stage-8:MAPRED' is a cross product +Warning: Map Join MAPJOIN[53][bigTable=?] in task 'Stage-7:MAPRED' is a cross product +Warning: Map Join MAPJOIN[43][bigTable=?] in task 'Stage-6:MAPRED' is a cross product +Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product PREHOOK: query: select * from orcsrc where not key in