diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 264c2b2..17fd55c 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -1265,7 +1265,15 @@ spark.query.files=add_part_multiple.q, \ statsfs.q, \ subquery_exists.q, \ subquery_in.q, \ + subquery_multi.q,\ subquery_multiinsert.q, \ + subquery_nested_subquery.q, \ + subquery_notin.q,\ + subquery_null_agg.q,\ + subquery_scalar.q,\ + subquery_select.q, \ + subquery_shared_alias.q, \ + subquery_views.q,\ table_access_keys_stats.q, \ temp_table.q, \ temp_table_gb1.q, \ diff --git a/ql/src/test/results/clientpositive/spark/subquery_multi.q.out b/ql/src/test/results/clientpositive/spark/subquery_multi.q.out new file mode 100644 index 0000000..f9b2c1b --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/subquery_multi.q.out @@ -0,0 +1,4129 @@ +PREHOOK: query: create table tnull(i int, c char(2)) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tnull +POSTHOOK: query: create table tnull(i int, c char(2)) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tnull +PREHOOK: query: insert into tnull values(NULL, NULL), (NULL, NULL) +PREHOOK: type: QUERY +PREHOOK: Output: default@tnull +POSTHOOK: query: insert into tnull values(NULL, NULL), (NULL, NULL) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@tnull +POSTHOOK: Lineage: tnull.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: tnull.i EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +PREHOOK: query: create table tempty(c char(2)) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tempty +POSTHOOK: query: create table tempty(c char(2)) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tempty +PREHOOK: query: CREATE TABLE part_null( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +ROW FORMAT DELIMITED FIELDS TERMINATED BY "," +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@part_null +POSTHOOK: query: CREATE TABLE part_null( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +ROW FORMAT DELIMITED FIELDS TERMINATED BY "," +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@part_null +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny_nulls.txt' overwrite into table part_null +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@part_null +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny_nulls.txt' overwrite into table part_null +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@part_null +PREHOOK: query: insert into part_null values(78487,NULL,'Manufacturer#6','Brand#52','LARGE BRUSHED BRASS', 23, 'MED BAG',1464.48,'hely blith') +PREHOOK: type: QUERY +PREHOOK: Output: default@part_null +POSTHOOK: query: insert into part_null values(78487,NULL,'Manufacturer#6','Brand#52','LARGE BRUSHED BRASS', 23, 'MED BAG',1464.48,'hely blith') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part_null +POSTHOOK: Lineage: part_null.p_brand SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col4, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_comment SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col9, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_container SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col7, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_mfgr SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col3, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_name SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_partkey EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_retailprice EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col8, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_size EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col6, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_type SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col5, type:string, comment:), ] +PREHOOK: query: explain select * from part_null where p_size IN (select p_size from part_null) AND p_brand IN (select p_brand from part_null) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_size IN (select p_size from part_null) AND p_brand IN (select p_brand from part_null) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_brand is not null and p_size is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col5 (type: int) + sort order: + + Map-reduce partition columns: _col5 (type: int) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + 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) + Map 4 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_size is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_size (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col5 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3581 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: 3581 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 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi 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: 3939 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3939 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part_null where p_size IN (select p_size from part_null) AND p_brand IN (select p_brand from part_null) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_size IN (select p_size from part_null) AND p_brand IN (select p_brand from part_null) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +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 +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +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 +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +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 +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +78487 NULL Manufacturer#6 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' 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 +POSTHOOK: 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) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 7 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 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: 3256 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) + Map 5 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Map 7 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + 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: 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) + Map 9 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 10 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3581 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3581 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 + Reduce Operator Tree: + 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: 3598 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: 3598 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 + Reduce Operator Tree: + 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: 3957 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col10 = 0) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) + Statistics: Num rows: 1 Data size: 3957 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: 3957 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3957 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 8 + 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 + 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) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' 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 +#### A masked pattern was here #### +POSTHOOK: 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) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' 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 +POSTHOOK: 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) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 7 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 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: 3256 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) + Map 5 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Map 7 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string) + outputColumnNames: p_type + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(p_type) + 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) + Map 9 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string) + outputColumnNames: p_type + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 10 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3581 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3581 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 + Reduce Operator Tree: + 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: 3598 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: 3598 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 + Reduce Operator Tree: + 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: 3957 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col10 = 0) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) + Statistics: Num rows: 1 Data size: 3957 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: 3957 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3957 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 8 + 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 + 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) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' 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 +#### A masked pattern was here #### +POSTHOOK: 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) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +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 +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +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 +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +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 +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 JOIN[28][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' 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 +POSTHOOK: 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) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 7 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 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: 3256 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) + Map 5 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_brand (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Map 7 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + 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: 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) + Map 9 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 10 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + 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: 3581 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3581 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 + Reduce Operator Tree: + 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: 3598 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: 3598 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 + Reduce Operator Tree: + 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: 3957 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col10 = 0) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) + Statistics: Num rows: 1 Data size: 3957 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: 3957 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3957 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 8 + 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 + 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) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[28][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' 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 +#### A masked pattern was here #### +POSTHOOK: 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) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +POSTHOOK: 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) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 9 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 1) + Reducer 8 <- Map 7 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3256 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) + Map 5 + Map Operator Tree: + TableScan + alias: tempty + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: c (type: char(2)) + outputColumnNames: c + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(), count(c) + 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) + Map 7 + Map Operator Tree: + TableScan + alias: tempty + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: c (type: char(2)) + outputColumnNames: c + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Group By Operator + keys: c (type: char(2)) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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: 0 Basic stats: PARTIAL Column stats: NONE + Map 9 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner 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: 3273 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: 3273 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) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 UDFToString(_col0) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + Statistics: Num rows: 1 Data size: 3600 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col12 is null and _col1 is not null and (_col10 >= _col9)) or (_col9 = 0)) (type: boolean) + Statistics: Num rows: 1 Data size: 3600 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: 3600 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: 3600 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 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi 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: 3960 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3960 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 + 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 + 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) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: char(2)) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: char(2)), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: UDFToString(_col0) (type: string) + sort order: + + Map-reduce partition columns: UDFToString(_col0) (type: string) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +PREHOOK: Input: default@tempty +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name NOT IN (select c from tempty) AND p_brand IN (select p_brand from part_null) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_null +POSTHOOK: Input: default@tempty +#### A masked pattern was here #### +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 +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +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 +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +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 +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 +78487 NULL Manufacturer#6 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND EXISTS (select c from tnull) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND EXISTS (select c from tnull) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 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: 3256 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) + Map 4 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: tnull + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: true (type: boolean) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE 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: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3581 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3581 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3586 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3586 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: boolean) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND EXISTS (select c from tnull) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_null +PREHOOK: Input: default@tnull +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND EXISTS (select c from tnull) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_null +POSTHOOK: Input: default@tnull +#### A masked pattern was here #### +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 +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +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 +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from part_null where p_size IN (select p_size from part_null) AND EXISTS (select c from tempty) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_size IN (select p_size from part_null) AND EXISTS (select c from tempty) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_size is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col5 (type: int) + sort order: + + Map-reduce partition columns: _col5 (type: int) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + 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) + Map 4 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_size is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_size (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: tempty + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: true (type: boolean) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col5 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3581 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3581 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3582 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3582 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: boolean) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from part_null where p_size IN (select p_size from part_null) AND EXISTS (select c from tempty) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_null +PREHOOK: Input: default@tempty +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_size IN (select p_size from part_null) AND EXISTS (select c from tempty) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_null +POSTHOOK: Input: default@tempty +#### A masked pattern was here #### +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 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: 3256 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) + Map 4 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: tempty + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: true (type: boolean) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3581 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3581 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 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 1 Data size: 3582 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col10 is null (type: boolean) + Statistics: Num rows: 1 Data size: 3582 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: 3582 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3582 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: boolean) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + value expressions: _col0 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_null +PREHOOK: Input: default@tempty +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_null +POSTHOOK: Input: default@tempty +#### A masked pattern was here #### +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 +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +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 +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +PREHOOK: query: explain select * from part_null where p_name IN ( select p_name from part where part.p_type = part_null.p_type) + AND p_brand NOT IN (select p_container from part where part.p_type = part_null.p_type + AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name IN ( select p_name from part where part.p_type = part_null.p_type) + AND p_brand NOT IN (select p_container from part where part.p_type = part_null.p_type + AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 12 <- Map 10 (PARTITION-LEVEL SORT, 2), Map 11 (PARTITION-LEVEL SORT, 2) + Reducer 13 <- Reducer 12 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 13 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 10 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 9 <- Reducer 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col4 (type: string), _col1 (type: string) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Map 10 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_brand (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 11 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string), p_type (type: string), p_container (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string), p_name (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string), p_type (type: string), p_container (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: string) + Reducer 12 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string), _col0 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col2 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reducer 13 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + 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: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col4 (type: string), _col1 (type: string) + 1 _col1 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 14 Data size: 1730 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: 14 Data size: 1730 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 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col12, _col13 + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string), _col4 (type: string) + sort order: ++ + Map-reduce partition columns: _col3 (type: string), _col4 (type: string) + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col12 (type: bigint), _col13 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col3 (type: string), _col4 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col12, _col13, _col16 + Statistics: Num rows: 16 Data size: 2093 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col12 = 0)) THEN (false) WHEN (_col12 is null) THEN (false) WHEN (_col16 is not null) THEN (true) WHEN (_col3 is null) THEN (null) WHEN ((_col13 < _col12)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 8 Data size: 1046 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: 8 Data size: 1046 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 1046 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reducer 8 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string), _col0 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col2) + keys: _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 28 Data size: 3461 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 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: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part_null where p_name IN ( select p_name from part where part.p_type = part_null.p_type) + AND p_brand NOT IN (select p_container from part where part.p_type = part_null.p_type + AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name IN ( select p_name from part where part.p_type = part_null.p_type) + AND p_brand NOT IN (select p_container from part where part.p_type = part_null.p_type + AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +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 +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +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 +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 +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +PREHOOK: query: explain select * from part_null where p_name IN ( select p_name from part) AND p_brand IN (select p_brand from part where part.p_type = part_null.p_type) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name IN ( select p_name from part) AND p_brand IN (select p_brand from part where part.p_type = part_null.p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 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: 3256 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) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string), p_type (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string), _col4 (type: string) + sort order: ++ + Map-reduce partition columns: _col3 (type: string), _col4 (type: string) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col3 (type: string), _col4 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + 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: NONE + 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: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part_null where p_name IN ( select p_name from part) AND p_brand IN (select p_brand from part where part.p_type = part_null.p_type) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name IN ( select p_name from part) AND p_brand IN (select p_brand from part where part.p_type = part_null.p_type) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +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 +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +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 +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 +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +PREHOOK: query: explain select * from part_null where p_name IN ( select p_name from part where part.p_type = part_null.p_type AND part.p_container=part_null.p_container) AND p_brand NOT IN (select p_container from part where part.p_type = part_null.p_type AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name IN ( select p_name from part where part.p_type = part_null.p_type AND part.p_container=part_null.p_container) AND p_brand NOT IN (select p_container from part where part.p_type = part_null.p_type AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 12 <- Map 10 (PARTITION-LEVEL SORT, 2), Map 11 (PARTITION-LEVEL SORT, 2) + Reducer 13 <- Reducer 12 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 13 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 10 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 9 <- Reducer 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: string), _col1 (type: string), _col6 (type: string) + sort order: +++ + Map-reduce partition columns: _col4 (type: string), _col1 (type: string), _col6 (type: string) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col7 (type: double), _col8 (type: string) + Map 10 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_brand (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 11 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string), p_type (type: string), p_container (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_container is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string), p_name (type: string), p_container (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), _col1 (type: string), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string), p_type (type: string), p_container (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: string) + Reducer 12 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string), _col0 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col1 (type: string), _col2 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reducer 13 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col4 (type: string), _col1 (type: string), _col6 (type: string) + 1 _col1 (type: string), _col0 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 14 Data size: 1730 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: 14 Data size: 1730 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 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col13, _col14 + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: string), _col3 (type: string) + sort order: ++ + Map-reduce partition columns: _col4 (type: string), _col3 (type: string) + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col13 (type: bigint), _col14 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: string), _col3 (type: string) + 1 _col1 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col13, _col14, _col17 + Statistics: Num rows: 16 Data size: 2093 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col13 = 0)) THEN (false) WHEN (_col13 is null) THEN (false) WHEN (_col17 is not null) THEN (true) WHEN (_col3 is null) THEN (null) WHEN ((_col14 < _col13)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 8 Data size: 1046 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: 8 Data size: 1046 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 1046 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string), _col2 (type: string) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reducer 8 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string), _col0 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col2) + keys: _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 28 Data size: 3461 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 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: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part_null where p_name IN ( select p_name from part where part.p_type = part_null.p_type AND part.p_container=part_null.p_container) AND p_brand NOT IN (select p_container from part where part.p_type = part_null.p_type AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name IN ( select p_name from part where part.p_type = part_null.p_type AND part.p_container=part_null.p_container) AND p_brand NOT IN (select p_container from part where part.p_type = part_null.p_type AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +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 +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type) AND p_brand NOT IN (select p_type from part where part.p_size = part_null.p_size) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type) AND p_brand NOT IN (select p_type from part where part.p_size = part_null.p_size) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 7 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col4 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col4 (type: string) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string), p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_size is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(p_type) + keys: p_size (type: int) + 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: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Map 9 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_size is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string), p_size (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 10 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string), _col4 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col5 (type: int) + sort order: + + Map-reduce partition columns: _col5 (type: int) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + 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) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col5 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col12, _col13 + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string), _col5 (type: int) + sort order: ++ + Map-reduce partition columns: _col3 (type: string), _col5 (type: int) + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col12 (type: bigint), _col13 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col3 (type: string), _col5 (type: int) + 1 _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col12, _col13, _col16 + Statistics: Num rows: 16 Data size: 2093 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col12 = 0)) THEN (false) WHEN (_col12 is null) THEN (false) WHEN (_col16 is not null) THEN (true) WHEN (_col3 is null) THEN (null) WHEN ((_col13 < _col12)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 8 Data size: 1046 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: 8 Data size: 1046 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 1046 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type) AND p_brand NOT IN (select p_type from part where part.p_size = part_null.p_size) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type) AND p_brand NOT IN (select p_type from part where part.p_size = part_null.p_size) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### 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 +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 +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +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 +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col4 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col4 (type: string) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_brand (type: string), p_type (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string), _col1 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string), p_brand (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string), _col4 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 30 Data size: 3807 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col2 (type: string), _col1 (type: string) + 1 _col1 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reducer 6 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type AND p_brand IN (select p_brand from part pp where part.p_type = pp.p_type)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +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 +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +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 +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +Warning: Shuffle Join JOIN[32][tables = [$hdt$_2, $hdt$_3]] in Work '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 + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber AND l_quantity NOT IN (select avg(l_quantity) from lineitem)) +PREHOOK: type: QUERY +POSTHOOK: 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 + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber AND l_quantity NOT IN (select avg(l_quantity) from lineitem)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 11 (GROUP, 1) + Reducer 12 <- Map 11 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 1), Reducer 10 (PARTITION-LEVEL SORT, 1) + Reducer 8 <- Reducer 12 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Map 11 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_quantity (type: double) + outputColumnNames: l_quantity + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(l_quantity) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: struct) + Map 5 + Map Operator Tree: + TableScan + alias: li + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((l_linenumber = 1) and l_partkey is not null) (type: boolean) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((l_shipmode = 'AIR') and l_linenumber is not null) (type: boolean) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), l_linenumber (type: int), l_quantity (type: double) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: double) + Reducer 10 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 12 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: double), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col1 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 55 Data size: 6598 Basic stats: COMPLETE Column stats: NONE + 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: 55 Data size: 6598 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col3 (type: int) + Reducer 4 + Reduce Operator Tree: + 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) + outputColumnNames: _col0, _col3 + Statistics: Num rows: 60 Data size: 7257 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 60 Data size: 7257 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 60 Data size: 7257 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 7 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col4, _col5 + Statistics: Num rows: 50 Data size: 10449 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: double) + sort order: + + Map-reduce partition columns: _col2 (type: double) + Statistics: Num rows: 50 Data size: 10449 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int), _col4 (type: bigint), _col5 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: double) + 1 _col0 (type: double) + outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col7 + Statistics: Num rows: 55 Data size: 11493 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col4 = 0)) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col2 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 28 Data size: 5850 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 5850 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int), _col1 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 5850 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 28 Data size: 5850 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[32][tables = [$hdt$_2, $hdt$_3]] in Work '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 + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber AND l_quantity NOT IN (select avg(l_quantity) from lineitem)) +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem +#### A masked pattern was here #### +POSTHOOK: 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 + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber AND l_quantity NOT IN (select avg(l_quantity) from lineitem)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem +#### A masked pattern was here #### +108570 8571 +4297 1798 +PREHOOK: query: explain +select key, value, count(*) +from src b +where b.key in (select key from src where src.value = b.value) +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' and exists (select * from src s2 where s1.value = s2.value) group by s1.key ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select key, value, count(*) +from src b +where b.key in (select key from src where src.value = b.value) +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' and exists (select * from src s2 where s1.value = s2.value) group by s1.key ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 2), Map 9 (PARTITION-LEVEL SORT, 2) + Reducer 8 <- Reducer 7 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + 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), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: value is not null (type: boolean) + 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 + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 9 + Map Operator Tree: + TableScan + alias: s2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: value is not null (type: boolean) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + 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: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col2 is not null (type: boolean) + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: bigint) + sort order: + + Map-reduce partition columns: _col2 (type: bigint) + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col2 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 302 Data size: 3213 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 7 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 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: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select key, value, count(*) +from src b +where b.key in (select key from src where src.value = b.value) +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' and exists (select * from src s2 where s1.value = s2.value) group by s1.key ) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select key, value, count(*) +from src b +where b.key in (select key from src where src.value = b.value) +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' and exists (select * from src s2 where s1.value = s2.value) group by s1.key ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +183 val_183 1 +266 val_266 1 +66 val_66 1 +472 val_472 1 +345 val_345 1 +452 val_452 1 +360 val_360 1 +136 val_136 1 +263 val_263 1 +85 val_85 1 +332 val_332 1 +375 val_375 1 +378 val_378 1 +339 val_339 1 +65 val_65 1 +17 val_17 1 +145 val_145 1 +189 val_189 1 +228 val_228 1 +449 val_449 1 +308 val_308 1 +158 val_158 1 +8 val_8 1 +467 val_467 1 +274 val_274 1 +402 val_402 1 +116 val_116 1 +286 val_286 1 +485 val_485 1 +28 val_28 1 +394 val_394 1 +291 val_291 1 +495 val_495 1 +235 val_235 1 +484 val_484 1 +258 val_258 1 +77 val_77 1 +150 val_150 1 +157 val_157 1 +87 val_87 1 +482 val_482 1 +493 val_493 1 +446 val_446 1 +302 val_302 1 +377 val_377 1 +9 val_9 1 +69 val_69 1 +180 val_180 1 +44 val_44 1 +78 val_78 1 +305 val_305 1 +80 val_80 1 +41 val_41 1 +163 val_163 1 +20 val_20 1 +296 val_296 1 +247 val_247 1 +448 val_448 1 +386 val_386 1 +196 val_196 1 +160 val_160 1 +436 val_436 1 +92 val_92 1 +400 val_400 1 +260 val_260 1 +379 val_379 1 +496 val_496 1 +487 val_487 1 +306 val_306 1 +19 val_19 1 +368 val_368 1 +74 val_74 1 +33 val_33 1 +57 val_57 1 +490 val_490 1 +389 val_389 1 +373 val_373 1 +497 val_497 1 +47 val_47 1 +170 val_170 1 +241 val_241 1 +356 val_356 1 +10 val_10 1 +190 val_190 1 +315 val_315 1 +30 val_30 1 +407 val_407 1 +54 val_54 1 +155 val_155 1 +248 val_248 1 +335 val_335 1 +131 val_131 1 +244 val_244 1 +105 val_105 1 +143 val_143 1 +494 val_494 1 +166 val_166 1 +366 val_366 1 +96 val_96 1 +292 val_292 1 +477 val_477 1 +43 val_43 1 +418 val_418 1 +202 val_202 1 +114 val_114 1 +178 val_178 1 +214 val_214 1 +252 val_252 1 +177 val_177 1 +435 val_435 1 +111 val_111 1 +257 val_257 1 +323 val_323 1 +392 val_392 1 +287 val_287 1 +341 val_341 1 +443 val_443 1 +53 val_53 1 +133 val_133 1 +457 val_457 1 +226 val_226 1 +364 val_364 1 +285 val_285 1 +481 val_481 1 +475 val_475 1 +421 val_421 1 +162 val_162 1 +310 val_310 1 +336 val_336 1 +393 val_393 1 +192 val_192 1 +126 val_126 1 +11 val_11 1 +455 val_455 1 +427 val_427 1 +4 val_4 1 +479 val_479 1 +289 val_289 1 +186 val_186 1 +249 val_249 1 +2 val_2 1 +181 val_181 1 +64 val_64 1 +437 val_437 1 +275 val_275 1 +362 val_362 1 +338 val_338 1 +419 val_419 1 +284 val_284 1 +460 val_460 1 +444 val_444 1 +156 val_156 1 +168 val_168 1 +365 val_365 1 +82 val_82 1 +374 val_374 1 +453 val_453 1 +153 val_153 1 +86 val_86 1 +34 val_34 1 +262 val_262 1 +470 val_470 1 +351 val_351 1 +201 val_201 1 +218 val_218 1 +483 val_483 1 +411 val_411 1 +27 val_27 1 +491 val_491 1 +222 val_222 1 +194 val_194 1 +283 val_283 1 +432 val_432 1 +382 val_382 2 +37 val_37 2 +95 val_95 2 +134 val_134 2 +172 val_172 2 +125 val_125 2 +58 val_58 2 +272 val_272 2 +103 val_103 2 +42 val_42 2 +353 val_353 2 +342 val_342 2 +120 val_120 2 +321 val_321 2 +209 val_209 2 +205 val_205 2 +84 val_84 2 +213 val_213 2 +331 val_331 2 +265 val_265 2 +317 val_317 2 +395 val_395 2 +195 val_195 2 +309 val_309 2 +462 val_462 2 +216 val_216 2 +224 val_224 2 +164 val_164 2 +149 val_149 2 +233 val_233 2 +367 val_367 2 +414 val_414 2 +239 val_239 2 +26 val_26 2 +307 val_307 2 +217 val_217 2 +191 val_191 2 +129 val_129 2 +223 val_223 2 +238 val_238 2 +24 val_24 2 +399 val_399 2 +459 val_459 2 +325 val_325 2 +255 val_255 2 +413 val_413 2 +51 val_51 2 +207 val_207 2 +281 val_281 2 +229 val_229 2 +221 val_221 2 +256 val_256 2 +219 val_219 2 +137 val_137 2 +282 val_282 2 +429 val_429 2 +104 val_104 2 +322 val_322 2 +12 val_12 2 +344 val_344 2 +404 val_404 2 +18 val_18 2 +83 val_83 2 +72 val_72 2 +200 val_200 2 +278 val_278 2 +15 val_15 2 +458 val_458 2 +152 val_152 2 +203 val_203 2 +237 val_237 2 +146 val_146 2 +165 val_165 2 +397 val_397 2 +280 val_280 2 +100 val_100 2 +118 val_118 2 +175 val_175 2 +113 val_113 2 +424 val_424 2 +197 val_197 2 +492 val_492 2 +333 val_333 2 +97 val_97 2 +76 val_76 2 +288 val_288 2 +179 val_179 2 +174 val_174 2 +67 val_67 2 +176 val_176 2 +478 val_478 2 +98 val_98 2 +242 val_242 2 +439 val_439 2 +463 val_463 2 +298 val_298 3 +193 val_193 3 +273 val_273 3 +316 val_316 3 +5 val_5 3 +369 val_369 3 +430 val_430 3 +70 val_70 3 +384 val_384 3 +431 val_431 3 +396 val_396 3 +480 val_480 3 +466 val_466 3 +498 val_498 3 +0 val_0 3 +318 val_318 3 +311 val_311 3 +327 val_327 3 +417 val_417 3 +454 val_454 3 +35 val_35 3 +90 val_90 3 +187 val_187 3 +208 val_208 3 +438 val_438 3 +403 val_403 3 +199 val_199 3 +128 val_128 3 +409 val_409 3 +167 val_167 3 +119 val_119 3 +Warning: Shuffle Join JOIN[13][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain select * from part where p_name IN (select p_name from part p where part.p_type <> '1') +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_name IN (select p_name from part p where part.p_type <> '1') +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col4 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col4 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_type <> '1') (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string), _col4 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 371 Data size: 90361 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 371 Data size: 90361 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 338 Data size: 82147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 338 Data size: 82147 Basic stats: COMPLETE Column stats: NONE + 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: 338 Data size: 82147 Basic stats: COMPLETE Column stats: NONE + Reducer 6 + 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: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[13][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select * from part where p_name IN (select p_name from part p where part.p_type <> '1') +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_name IN (select p_name from part p where part.p_type <> '1') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +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 +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +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 +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +PREHOOK: query: insert into tnull values(1, 'c') +PREHOOK: type: QUERY +PREHOOK: Output: default@tnull +POSTHOOK: query: insert into tnull values(1, 'c') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@tnull +POSTHOOK: Lineage: tnull.c EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: tnull.i EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +POSTHOOK: query: explain select * from part where p_partkey = 3 OR p_size NOT IN (select i from tnull) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: tnull + Statistics: Num rows: 3 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 3 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(i) + 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) + Map 6 + Map Operator Tree: + TableScan + alias: tnull + Statistics: Num rows: 3 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 3 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner 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: 3589 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col5 (type: int) + sort order: + + Map-reduce partition columns: _col5 (type: int) + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + 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) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer 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 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col0 = 3) or CASE WHEN ((_col9 = 0)) 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: 28 Data size: 3947 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: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3947 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 + 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 + 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) + Reducer 7 + 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: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +PREHOOK: Input: default@tnull +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_partkey = 3 OR p_size NOT IN (select i from tnull) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@tnull +#### A masked pattern was here #### +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain select count(*) from src + where src.key in (select key from src s1 where s1.key > '9') + or src.value is not null + or exists(select key from src) +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(*) from src + where src.key in (select key from src s1 where s1.key > '9') + or src.value is not null + or exists(select key from src) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 11 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + 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 + sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 10 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: true (type: boolean) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE 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: 4 Basic stats: COMPLETE Column stats: COMPLETE + Map 6 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Map 8 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: boolean) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 500 Data size: 9812 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: 9812 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col2, _col4 + Statistics: Num rows: 550 Data size: 10793 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 550 Data size: 10793 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: bigint), _col4 (type: boolean) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col4, _col5 + Statistics: Num rows: 550 Data size: 13543 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col4 is not null and (_col2 <> 0)) or _col1 is not null or _col5 is not null) (type: boolean) + Statistics: Num rows: 550 Data size: 13543 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 550 Data size: 13543 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 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 7 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 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: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select count(*) from src + where src.key in (select key from src s1 where s1.key > '9') + or src.value is not null + or exists(select key from src) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from src + where src.key in (select key from src s1 where s1.key > '9') + or src.value is not null + or exists(select key from src) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +500 +PREHOOK: query: drop table tnull +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tnull +PREHOOK: Output: default@tnull +POSTHOOK: query: drop table tnull +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tnull +POSTHOOK: Output: default@tnull +PREHOOK: query: drop table tempty +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tempty +PREHOOK: Output: default@tempty +POSTHOOK: query: drop table tempty +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tempty +POSTHOOK: Output: default@tempty +PREHOOK: query: drop table part_null +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@part_null +PREHOOK: Output: default@part_null +POSTHOOK: query: drop table part_null +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@part_null +POSTHOOK: Output: default@part_null diff --git a/ql/src/test/results/clientpositive/spark/subquery_nested_subquery.q.out b/ql/src/test/results/clientpositive/spark/subquery_nested_subquery.q.out new file mode 100644 index 0000000..15f33f0 --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/subquery_nested_subquery.q.out @@ -0,0 +1,38 @@ +PREHOOK: query: select * +from part x +where x.p_name in (select y.p_name from part y where exists (select z.p_name from part z where y.p_name = z.p_name)) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * +from part x +where x.p_name in (select y.p_name from part y where exists (select z.p_name from part z where y.p_name = z.p_name)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +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 +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +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 +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref diff --git a/ql/src/test/results/clientpositive/spark/subquery_notin.q.out b/ql/src/test/results/clientpositive/spark/subquery_notin.q.out new file mode 100644 index 0000000..d7b9a41 --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/subquery_notin.q.out @@ -0,0 +1,7722 @@ +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain +select * +from src +where src.key not in + ( select key from src s1 + where s1.key > '2' + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * +from src +where src.key not in + ( select key from src s1 + where s1.key > '2' + ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + 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 + sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '2') (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) + Map 6 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '2') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 500 Data size: 13812 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: 13812 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: bigint), _col3 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + 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 = 0) 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 + Statistics: Num rows: 366 Data size: 10110 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 + 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 + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 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: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * +from src +where src.key not in ( select key from src s1 where s1.key > '2') +order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from src +where src.key not in ( select key from src s1 where s1.key > '2') +order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +10 val_10 +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 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +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 +PREHOOK: query: explain +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 2) + Reducer 8 <- Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 9 <- Reducer 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col2 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_mfgr is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + value expressions: p_name (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4, _col5 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: int), _col4 (type: bigint), _col5 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col8 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col4 = 0)) THEN (false) WHEN (_col4 is null) THEN (false) WHEN (_col8 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 1903 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 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col5 + name: rank + window function: GenericUDAFRankEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (rank_window_0 <= 2) (type: boolean) + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + keys: _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 968 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: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 4 Data size: 484 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: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col5 + name: rank + window function: GenericUDAFRankEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (rank_window_0 <= 2) (type: boolean) + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + 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: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + 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: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +order by p_mfgr, b.p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +order by p_mfgr, b.p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 +Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond antique violet turquoise frosted 40 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#3 almond antique metallic orange dim 19 +Manufacturer#3 almond antique olive coral navajo 45 +Manufacturer#4 almond antique violet mint lemon 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 +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 JOIN[33][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain +select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 1) + Reducer 8 <- Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 9 <- Reducer 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 26 Data size: 5357 Basic stats: COMPLETE Column stats: NONE + 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: 5357 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 UDFToDouble(_col1) (type: double) + 1 _col0 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Statistics: Num rows: 28 Data size: 5892 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = 0) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 18 Data size: 3787 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 3787 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 18 Data size: 3787 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 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col5 + name: rank + window function: GenericUDAFRankEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (rank_window_0 <= 2) (type: boolean) + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col5 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: struct) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col5 + name: rank + window function: GenericUDAFRankEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (rank_window_0 <= 2) (type: boolean) + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col5 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: struct) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: double), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[33][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +order by p_name, p_size +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +order by p_name, p_size +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +almond antique blue firebrick mint 31 +almond antique burnished rose metallic 2 +almond antique burnished rose metallic 2 +almond antique chartreuse khaki white 17 +almond antique chartreuse lavender yellow 34 +almond antique forest lavender goldenrod 14 +almond antique gainsboro frosted violet 10 +almond antique metallic orange dim 19 +almond antique misty red olive 1 +almond antique olive coral navajo 45 +almond antique sky peru orange 2 +almond antique violet chocolate turquoise 14 +almond antique violet mint lemon 39 +almond antique violet turquoise frosted 40 +almond aquamarine burnished black steel 28 +almond aquamarine dodger light gainsboro 46 +almond aquamarine floral ivory bisque 27 +almond aquamarine midnight light salmon 2 +almond aquamarine pink moccasin thistle 42 +almond aquamarine rose maroon antique 25 +almond aquamarine sandy cyan gainsboro 18 +almond aquamarine yellow dodger mint 7 +almond azure aquamarine papaya violet 12 +almond azure blanched chiffon midnight 23 +PREHOOK: query: explain +select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Reducer 9 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 12 <- Map 11 (PARTITION-LEVEL SORT, 2) + Reducer 13 <- Reducer 12 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 13 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 11 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Reducer 6 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 9 <- Map 11 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col2 (type: int) + Map 11 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_mfgr is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Reducer 10 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 2 Data size: 242 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: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 12 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col5 + name: rank + window function: GenericUDAFRankEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (rank_window_0 <= 2) (type: boolean) + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string), _col5 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 968 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: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Reducer 13 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: int) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col4, true) > 0) (type: boolean) + Statistics: Num rows: 9 Data size: 1112 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 9 Data size: 1112 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: 9 Data size: 1112 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col2 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col6, _col7 + Statistics: Num rows: 9 Data size: 1223 Basic stats: COMPLETE Column stats: NONE + 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: 9 Data size: 1223 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col6 (type: bigint), _col7 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string), _col2 (type: int) + 1 _col1 (type: string), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col10 + Statistics: Num rows: 9 Data size: 1345 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col6 = 0)) THEN (false) WHEN (_col6 is null) THEN (false) WHEN (_col10 is not null) THEN (true) WHEN (_col2 is null) THEN (null) WHEN ((_col7 < _col6)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 5 Data size: 747 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 747 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 747 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 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col5 + name: rank + window function: GenericUDAFRankEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (rank_window_0 <= 2) (type: boolean) + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 8 Data size: 968 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: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 242 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: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col5 + name: rank + window function: GenericUDAFRankEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (rank_window_0 <= 2) (type: boolean) + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string), _col5 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 968 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: 8 Data size: 968 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond antique violet turquoise frosted 40 +Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#3 almond antique metallic orange dim 19 +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 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#3 almond antique forest lavender goldenrod 14 +Manufacturer#3 almond antique olive coral navajo 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 +Manufacturer#4 almond antique violet mint lemon 39 +Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select li.l_partkey, count(*) +from lineitem li +where li.l_linenumber = 1 and + li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') +group by li.l_partkey +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem +#### A masked pattern was here #### +POSTHOOK: query: select li.l_partkey, count(*) +from lineitem li +where li.l_linenumber = 1 and + li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') +group by li.l_partkey +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem +#### A masked pattern was here #### +22630 1 +7068 1 +88362 1 +123076 1 +139636 1 +106170 1 +182052 1 +59694 1 +175839 1 +21636 1 +119767 1 +119477 1 +88035 1 +450 1 +61931 1 +85951 1 +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * +from src +where not src.key in ( select key from src s1 where s1.key > '2') +order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from src +where not src.key in ( select key from src s1 where s1.key > '2') +order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +10 val_10 +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 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +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 +PREHOOK: query: create view T1_v as +select key from src where key <'11' +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@T1_v +POSTHOOK: query: create view T1_v as +select key from src where key <'11' +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1_v +POSTHOOK: Lineage: T1_v.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: create view T2_v as +select case when key > '104' then null else key end as key from T1_v +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@src +PREHOOK: Input: default@t1_v +PREHOOK: Output: database:default +PREHOOK: Output: default@T2_v +POSTHOOK: query: create view T2_v as +select case when key > '104' then null else key end as key from T1_v +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@src +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 JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain +select * +from T1_v where T1_v.key not in (select T2_v.key from T2_v) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * +from T1_v where T1_v.key not in (select T2_v.key from T2_v) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + properties: + insideView TRUE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < '11') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: src + properties: + insideView TRUE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < '11') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: CASE WHEN ((key > '104')) THEN (null) ELSE (key) END (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + 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) + Map 6 + Map Operator Tree: + TableScan + alias: src + properties: + insideView TRUE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key < '11') and CASE WHEN ((key > '104')) THEN (null) ELSE ((key < '11')) END) (type: boolean) + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: CASE WHEN ((key > '104')) THEN (null) ELSE (key) END (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 83 Data size: 881 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: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 166 Data size: 4585 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: 166 Data size: 4585 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 182 Data size: 5043 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 = 0) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Statistics: Num rows: 121 Data size: 3352 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 121 Data size: 3352 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 121 Data size: 3352 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 + 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 + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 41 Data size: 435 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: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +PREHOOK: Input: default@src +PREHOOK: Input: default@t1_v +PREHOOK: Input: default@t2_v +#### A masked pattern was here #### +POSTHOOK: query: select * +from T1_v where T1_v.key not in (select T2_v.key from T2_v) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@t1_v +POSTHOOK: Input: default@t2_v +#### A masked pattern was here #### +PREHOOK: query: explain select * from part where p_brand <> 'Brand#14' AND p_size NOT IN (select (p_size*p_size) from part p where p.p_type = part.p_type ) AND p_size <> 340 +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_brand <> 'Brand#14' AND p_size NOT IN (select (p_size*p_size) from part p where p.p_type = part.p_type ) AND p_size <> 340 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((p_brand <> 'Brand#14') and (p_size <> 340)) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), (p_size * p_size) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (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: bigint), _col2 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: (p_size * p_size) (type: int), p_type (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col1 (type: string), _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: string), _col5 (type: int) + sort order: ++ + Map-reduce partition columns: _col4 (type: string), _col5 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: bigint), _col11 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: string), _col5 (type: int) + 1 _col1 (type: string), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col14 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col10 = 0)) THEN (false) WHEN (_col10 is null) THEN (false) WHEN (_col14 is not null) THEN (true) WHEN (_col5 is null) THEN (null) WHEN ((_col11 < _col10)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 15 Data size: 1903 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: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 1903 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(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 + 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: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part where p_brand <> 'Brand#14' AND p_size NOT IN (select (p_size*p_size) from part p where p.p_type = part.p_type ) AND p_size <> 340 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_brand <> 'Brand#14' AND p_size NOT IN (select (p_size*p_size) from part p where p.p_type = part.p_type ) AND p_size <> 340 +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 +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +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 +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +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 JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 6 <- Reducer 5 (GROUP, 1) + Reducer 8 <- Map 4 (GROUP, 2) + Reducer 9 <- Reducer 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: p_type, p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner 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: 3589 Basic stats: COMPLETE Column stats: NONE + 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: 3589 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), _col9 (type: bigint), _col10 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer 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 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0)) (type: boolean) + Statistics: Num rows: 18 Data size: 2537 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: 18 Data size: 2537 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 18 Data size: 2537 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + 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) + Reducer 6 + 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 + 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) + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col1 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +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 +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +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 +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +POSTHOOK: query: explain select * from part where (p_partkey*p_size) NOT IN (select min(p_partkey) from part group by p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 6 <- Reducer 5 (GROUP, 1) + Reducer 8 <- Map 4 (GROUP, 2) + Reducer 9 <- Reducer 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_type (type: string) + outputColumnNames: p_partkey, p_type + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_partkey) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner 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: 3589 Basic stats: COMPLETE Column stats: NONE + 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: 3589 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), _col9 (type: bigint), _col10 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer 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 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col12 is null and _col0 is not null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0)) (type: boolean) + Statistics: Num rows: 18 Data size: 2537 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: 18 Data size: 2537 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 18 Data size: 2537 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + 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) + Reducer 6 + 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 + 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) + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col1 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +#### A masked pattern was here #### +POSTHOOK: query: select * from part where (p_partkey*p_size) NOT IN (select min(p_partkey) from part group by p_type) +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 +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 +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +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 +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +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 +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 +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 +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +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 +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +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 +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 +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 +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 +POSTHOOK: 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) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 7 (GROUP, 2) + Reducer 9 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: e + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 10 + Map Operator Tree: + TableScan + alias: e + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_size (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(p_partkey) + keys: p_name (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: bigint), _col2 (type: bigint) + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_partkey (type: int), p_name (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: (_col0 + 100) (type: int) + sort order: + + Map-reduce partition columns: (_col0 + 100) (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: int) + 1 _col1 (type: string), _col3 (type: int) + outputColumnNames: _col1, _col3, _col4, _col7 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN ((_col1 + 100) is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(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 + 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: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: boolean) + Reducer 9 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 (_col0 + 100) (type: int) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col3 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col3 (type: int) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 +#### A masked pattern was here #### +POSTHOOK: query: 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) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +26 +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +POSTHOOK: query: explain select * from part where floor(p_retailprice) NOT IN (select floor(min(p_retailprice)) from part group by p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 6 <- Reducer 5 (GROUP, 1) + Reducer 8 <- Map 4 (GROUP, 2) + Reducer 9 <- Reducer 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_retailprice (type: double) + outputColumnNames: p_type, p_retailprice + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_retailprice) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner 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: 3589 Basic stats: COMPLETE Column stats: NONE + 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: 3589 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), _col9 (type: bigint), _col10 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer 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 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col12 is null and floor(_col7) is not null and (_col10 >= _col9)) or (_col9 = 0)) (type: boolean) + Statistics: Num rows: 18 Data size: 2537 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: 18 Data size: 2537 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 18 Data size: 2537 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: floor(_col1) (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + 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) + Reducer 6 + 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 + 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) + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: floor(_col1) (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: bigint) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +#### A masked pattern was here #### +POSTHOOK: query: select * from part where floor(p_retailprice) NOT IN (select floor(min(p_retailprice)) from part group by p_type) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +PREHOOK: query: explain select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (PARTITION-LEVEL SORT, 2), Reducer 13 (PARTITION-LEVEL SORT, 2) + Reducer 11 <- Reducer 10 (GROUP, 2) + Reducer 13 <- Map 12 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 2) + Reducer 8 <- Map 12 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col5 (type: int) + sort order: + + Map-reduce partition columns: _col5 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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) + Map 12 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_size (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col2 (type: int), _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Map 9 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col2 (type: int), _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 10 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col2 (type: int), _col0 (type: int) + 1 _col0 (type: int), (_col0 + 121150) (type: int) + outputColumnNames: _col1, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col1 (type: string), _col3 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + 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: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 13 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), (_col0 + 121150) (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), (_col0 + 121150) (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col5 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col5 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col5 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: bigint), _col11 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string), _col5 (type: int) + 1 _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col14 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col10 = 0)) THEN (false) WHEN (_col10 is null) THEN (false) WHEN (_col14 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col11 < _col10)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 15 Data size: 1903 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: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 1903 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col2 (type: int), _col0 (type: int) + 1 _col0 (type: int), (_col0 + 121150) (type: int) + outputColumnNames: _col1, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col3 (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), (_col0 + 121150) (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), (_col0 + 121150) (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey ) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +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 +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +PREHOOK: query: explain select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_partkey= p.p_partkey ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_partkey= p.p_partkey ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col5 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col5 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_partkey is not null and p_size is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(p_name) + keys: p_partkey (type: int), p_size (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint), _col3 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_partkey is not null and p_size is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_partkey (type: int), p_name (type: string), p_size (type: int) + 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: int), _col1 (type: string), _col2 (type: int) + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string), _col2 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int), _col5 (type: int) + 1 _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col11, _col12 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int) + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string), _col5 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col11 (type: bigint), _col12 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int), _col1 (type: string), _col5 (type: int) + 1 _col1 (type: int), _col0 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col11, _col12, _col16 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col11 = 0)) THEN (false) WHEN (_col11 is null) THEN (false) WHEN (_col16 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col12 < _col11)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 15 Data size: 1903 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: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 1903 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint), _col3 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string), KEY._col2 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: int), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int), _col0 (type: string), _col2 (type: int) + sort order: +++ + Map-reduce partition columns: _col1 (type: int), _col0 (type: string), _col2 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_partkey= p.p_partkey ) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_partkey= p.p_partkey ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +PREHOOK: query: explain select p_name from (select p_name, p_type, p_brand as brand from part) fpart where fpart.p_type NOT IN (select p_type+2 from part where part.p_brand = fpart.brand) +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_name from (select p_name, p_type, p_brand as brand from part) fpart where fpart.p_type NOT IN (select p_type+2 from part where part.p_brand = fpart.brand) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_type (type: string), p_brand (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string), (UDFToDouble(p_type) + 2.0) (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (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: bigint), _col2 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: (UDFToDouble(p_type) + 2.0) (type: double), p_brand (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: double), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: double), _col1 (type: string) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4, _col5 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(_col1) (type: double), _col2 (type: string) + sort order: ++ + Map-reduce partition columns: UDFToDouble(_col1) (type: double), _col2 (type: string) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 UDFToDouble(_col1) (type: double), _col2 (type: string) + 1 _col0 (type: double), _col1 (type: string) + outputColumnNames: _col0, _col1, _col4, _col5, _col8 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col4 = 0)) THEN (false) WHEN (_col4 is null) THEN (false) WHEN (_col8 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 1903 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(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 + 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: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: double), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: double), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: double), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: double), _col1 (type: string) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select p_name from (select p_name, p_type, p_brand as brand from part) fpart where fpart.p_type NOT IN (select p_type+2 from part where part.p_brand = fpart.brand) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_name from (select p_name, p_type, p_brand as brand from part) fpart where fpart.p_type NOT IN (select p_type+2 from part where part.p_brand = fpart.brand) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +PREHOOK: query: explain select p_name from (select p_name, p_type, p_size+1 as size from part) fpart where fpart.p_type NOT IN (select p_type from part where (part.p_size+1) = fpart.size) +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_name from (select p_name, p_type, p_size+1 as size from part) fpart where fpart.p_type NOT IN (select p_type from part where (part.p_size+1) = fpart.size) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (PARTITION-LEVEL SORT, 2), Reducer 13 (PARTITION-LEVEL SORT, 2) + Reducer 11 <- Reducer 10 (GROUP, 2) + Reducer 13 <- Map 12 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 2) + Reducer 8 <- Map 12 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_type (type: string), (p_size + 1) (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 12 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: (p_size + 1) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: (_col1 + 1) (type: int) + sort order: + + Map-reduce partition columns: (_col1 + 1) (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 9 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: (_col1 + 1) (type: int) + sort order: + + Map-reduce partition columns: (_col1 + 1) (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 10 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 (_col1 + 1) (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col2 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + 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: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 13 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4, _col5 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col4 (type: bigint), _col5 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string), _col2 (type: int) + 1 _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col4, _col5, _col8 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col4 = 0)) THEN (false) WHEN (_col4 is null) THEN (false) WHEN (_col8 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 1903 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 (_col1 + 1) (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + keys: _col2 (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select p_name from (select p_name, p_type, p_size+1 as size from part) fpart where fpart.p_type NOT IN (select p_type from part where (part.p_size+1) = fpart.size+1) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_name from (select p_name, p_type, p_size+1 as size from part) fpart where fpart.p_type NOT IN (select p_type from part where (part.p_size+1) = fpart.size+1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +almond aquamarine yellow dodger mint +almond aquamarine dodger light gainsboro +almond antique violet turquoise frosted +almond aquamarine pink moccasin thistle +almond antique salmon chartreuse burlywood +almond antique burnished rose metallic +almond antique burnished rose metallic +almond antique gainsboro frosted violet +almond azure aquamarine papaya violet +almond aquamarine sandy cyan gainsboro +almond antique olive coral navajo +almond antique misty red olive +almond azure blanched chiffon midnight +almond antique chartreuse lavender yellow +almond antique violet chocolate turquoise +almond antique metallic orange dim +almond aquamarine midnight light salmon +almond antique chartreuse khaki white +almond antique blue firebrick mint +almond antique forest lavender goldenrod +almond antique violet mint lemon +almond antique sky peru orange +almond aquamarine floral ivory bisque +almond aquamarine rose maroon antique +almond aquamarine burnished black steel +almond antique medium spring khaki +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +POSTHOOK: 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 ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 2) + Reducer 5 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + 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 + sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 10 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = '90') (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: '90' (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 6 + 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) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 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) + Map 8 + 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) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 500 Data size: 13812 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: 500 Data size: 13812 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col2 (type: bigint), _col3 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (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 = 0) or (_col5 is null and _col1 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) + outputColumnNames: _col0 + Statistics: Num rows: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 366 Data size: 10110 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: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 183 Data size: 5055 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 183 Data size: 5055 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: bigint) + sort order: + + Map-reduce partition columns: _col1 (type: bigint) + Statistics: Num rows: 183 Data size: 5055 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 201 Data size: 5560 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 201 Data size: 5560 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 7 + 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 + 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) + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + 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 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +#### A masked pattern was here #### +POSTHOOK: 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 ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +273 3 +454 3 +311 3 +128 3 +396 3 +316 3 +384 3 +369 3 +417 3 +298 3 +208 3 +199 3 +430 3 +187 3 +193 3 +318 3 +327 3 +5 3 +119 3 +466 3 +70 3 +167 3 +403 3 +480 3 +0 3 +409 3 +35 3 +90 3 +438 3 +431 3 +498 3 +PREHOOK: query: explain select key, count(*) from src where value NOT IN (select concat('v', value) from src sc where sc.key = src.key ) group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, count(*) from src where value NOT IN (select concat('v', value) from src sc where sc.key = src.key ) group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 2) + Reducer 5 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (GROUP, 2) + Reducer 9 <- Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + 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) + Map 10 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = '90') (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: '90' (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: sc + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), concat('v', value) (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + 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: bigint), _col2 (type: bigint) + Map 8 + Map Operator Tree: + TableScan + alias: sc + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: concat('v', value) (type: string), key (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col1 (type: string), _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + 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: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col1 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1, _col3, _col4, _col7 + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 303 Data size: 3218 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 303 Data size: 3218 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 303 Data size: 3218 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: 303 Data size: 3218 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 151 Data size: 1603 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 151 Data size: 1603 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: bigint) + sort order: + + Map-reduce partition columns: _col1 (type: bigint) + Statistics: Num rows: 151 Data size: 1603 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 166 Data size: 1763 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 7 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select key, count(*) from src where value NOT IN (select concat('v', value) from src sc where sc.key = src.key ) 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 +#### A masked pattern was here #### +POSTHOOK: query: select key, count(*) from src where value NOT IN (select concat('v', value) from src sc where sc.key = src.key ) group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +273 3 +454 3 +311 3 +128 3 +396 3 +316 3 +384 3 +369 3 +417 3 +298 3 +208 3 +199 3 +430 3 +187 3 +193 3 +318 3 +327 3 +5 3 +119 3 +466 3 +70 3 +167 3 +403 3 +480 3 +0 3 +409 3 +35 3 +90 3 +438 3 +431 3 +498 3 +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +POSTHOOK: 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Reducer 9 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (SORT, 1) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 7 <- Reducer 6 (GROUP, 1) + Reducer 9 <- Map 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: p_type, p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 10 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner 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: 3589 Basic stats: COMPLETE Column stats: NONE + 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: 3589 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), _col9 (type: bigint), _col10 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer 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 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0)) (type: boolean) + Statistics: Num rows: 18 Data size: 2537 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: 18 Data size: 2537 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Statistics: Num rows: 18 Data size: 2537 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 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 18 Data size: 2537 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 18 Data size: 2537 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + 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) + Reducer 7 + 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 + 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) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col1 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +#### A masked pattern was here #### +POSTHOOK: query: select * from part where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +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 +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +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 +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 JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +POSTHOOK: 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Reducer 9 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (SORT, 1) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 7 <- Reducer 6 (GROUP, 1) + Reducer 9 <- Map 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: p_type, p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 10 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner 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: 3589 Basic stats: COMPLETE Column stats: NONE + 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: 3589 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), _col9 (type: bigint), _col10 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer 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 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col12 is null and _col5 is not null and (_col10 >= _col9)) or (_col9 = 0)) (type: boolean) + Statistics: Num rows: 18 Data size: 2537 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: 18 Data size: 2537 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string), _col0 (type: int) + sort order: ++ + Statistics: Num rows: 18 Data size: 2537 Basic stats: COMPLETE Column stats: NONE + 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 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 18 Data size: 2537 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 4 + Statistics: Num rows: 4 Data size: 560 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 560 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + 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) + Reducer 7 + 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 + 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) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col1 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: 4 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +#### A masked pattern was here #### +POSTHOOK: 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 +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 +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 JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +POSTHOOK: query: explain select * from src where key NOT IN (select p_name from part UNION ALL select p_brand from part) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1), Map 6 (GROUP, 1) + Reducer 8 <- Map 7 (GROUP, 2), Map 9 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + 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 + sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + 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) + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + 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) + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 52 Data size: 6294 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: 52 Data size: 6294 Basic stats: COMPLETE Column stats: NONE + Map 9 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 52 Data size: 6294 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: 52 Data size: 6294 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 500 Data size: 13812 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: 13812 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: bigint), _col3 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + 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 = 0) 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 + Statistics: Num rows: 366 Data size: 10110 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 + 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 + 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) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + 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: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' 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 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from src where key NOT IN (select p_name from part UNION ALL select p_brand from part) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@src +#### A masked pattern was here #### +10 val_10 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +111 val_111 +114 val_114 +116 val_116 +120 val_120 +120 val_120 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +15 val_15 +15 val_15 +152 val_152 +152 val_152 +155 val_155 +157 val_157 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +17 val_17 +170 val_170 +172 val_172 +172 val_172 +178 val_178 +179 val_179 +179 val_179 +181 val_181 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +19 val_19 +191 val_191 +191 val_191 +192 val_192 +194 val_194 +195 val_195 +195 val_195 +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 +203 val_203 +203 val_203 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +218 val_218 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +228 val_228 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +239 val_239 +239 val_239 +24 val_24 +24 val_24 +241 val_241 +244 val_244 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +26 val_26 +26 val_26 +262 val_262 +265 val_265 +265 val_265 +266 val_266 +27 val_27 +273 val_273 +273 val_273 +273 val_273 +275 val_275 +278 val_278 +278 val_278 +28 val_28 +280 val_280 +280 val_280 +282 val_282 +282 val_282 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +291 val_291 +292 val_292 +298 val_298 +298 val_298 +298 val_298 +30 val_30 +302 val_302 +305 val_305 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +321 val_321 +321 val_321 +323 val_323 +33 val_33 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +344 val_344 +344 val_344 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +37 val_37 +37 val_37 +373 val_373 +377 val_377 +378 val_378 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +392 val_392 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +402 val_402 +404 val_404 +404 val_404 +407 val_407 +411 val_411 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +42 val_42 +42 val_42 +424 val_424 +424 val_424 +43 val_43 +430 val_430 +430 val_430 +430 val_430 +432 val_432 +436 val_436 +437 val_437 +444 val_444 +448 val_448 +449 val_449 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +457 val_457 +459 val_459 +459 val_459 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +47 val_47 +477 val_477 +479 val_479 +481 val_481 +482 val_482 +483 val_483 +485 val_485 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +57 val_57 +65 val_65 +67 val_67 +67 val_67 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +8 val_8 +80 val_80 +82 val_82 +85 val_85 +86 val_86 +87 val_87 +9 val_9 +92 val_92 +96 val_96 +97 val_97 +97 val_97 +0 val_0 +0 val_0 +0 val_0 +105 val_105 +11 val_11 +113 val_113 +113 val_113 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +125 val_125 +125 val_125 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +150 val_150 +153 val_153 +156 val_156 +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 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +18 val_18 +18 val_18 +180 val_180 +183 val_183 +186 val_186 +190 val_190 +193 val_193 +193 val_193 +193 val_193 +196 val_196 +197 val_197 +197 val_197 +205 val_205 +205 val_205 +209 val_209 +209 val_209 +217 val_217 +217 val_217 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +226 val_226 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +238 val_238 +238 val_238 +242 val_242 +242 val_242 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +260 val_260 +263 val_263 +272 val_272 +272 val_272 +274 val_274 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +281 val_281 +281 val_281 +283 val_283 +289 val_289 +296 val_296 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +315 val_315 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +322 val_322 +322 val_322 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +34 val_34 +341 val_341 +342 val_342 +342 val_342 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +35 val_35 +35 val_35 +35 val_35 +351 val_351 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +374 val_374 +375 val_375 +379 val_379 +382 val_382 +382 val_382 +389 val_389 +393 val_393 +394 val_394 +399 val_399 +399 val_399 +4 val_4 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +403 val_403 +403 val_403 +403 val_403 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +41 val_41 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +421 val_421 +427 val_427 +429 val_429 +429 val_429 +431 val_431 +431 val_431 +431 val_431 +435 val_435 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +44 val_44 +443 val_443 +446 val_446 +452 val_452 +455 val_455 +458 val_458 +458 val_458 +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 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +478 val_478 +478 val_478 +480 val_480 +480 val_480 +480 val_480 +484 val_484 +487 val_487 +491 val_491 +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 +58 val_58 +58 val_58 +64 val_64 +66 val_66 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +77 val_77 +78 val_78 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +90 val_90 +90 val_90 +90 val_90 +95 val_95 +95 val_95 +98 val_98 +98 val_98 +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 +POSTHOOK: 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) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 7 (GROUP, 2) + Reducer 9 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: e + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 10 + Map Operator Tree: + TableScan + alias: e + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_size (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(p_type) + keys: p_brand (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: bigint), _col2 (type: bigint) + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_brand is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_brand (type: string), p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble((_col0 + 100)) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble((_col0 + 100)) (type: double) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: int) + 1 _col1 (type: string), _col3 (type: int) + outputColumnNames: _col1, _col3, _col4, _col7 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN ((_col1 + 100) is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(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 + 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: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(_col0) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col0) (type: double) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: boolean) + Reducer 9 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 UDFToDouble((_col0 + 100)) (type: double) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col3 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col3 (type: int) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 +#### A masked pattern was here #### +POSTHOOK: query: 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) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +26 +PREHOOK: query: CREATE TABLE t1 (c1 INT, c2 CHAR(100)) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: CREATE TABLE t1 (c1 INT, c2 CHAR(100)) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +PREHOOK: query: INSERT INTO t1 VALUES (null,null), (1,''), (2,'abcde'), (100,'abcdefghij') +PREHOOK: type: QUERY +PREHOOK: Output: default@t1 +POSTHOOK: query: INSERT INTO t1 VALUES (null,null), (1,''), (2,'abcde'), (100,'abcdefghij') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@t1 +POSTHOOK: Lineage: t1.c1 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: t1.c2 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: CREATE TABLE t2 (c1 INT) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t2 +POSTHOOK: query: CREATE TABLE t2 (c1 INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t2 +PREHOOK: query: INSERT INTO t2 VALUES (null), (2), (100) +PREHOOK: type: QUERY +PREHOOK: Output: default@t2 +POSTHOOK: query: INSERT INTO t2 VALUES (null), (2), (100) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2.c1 EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain SELECT c1 FROM t1 WHERE c1 NOT IN (SELECT c1 FROM t2) +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT c1 FROM t1 WHERE c1 NOT IN (SELECT c1 FROM t2) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: c1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: t2 + 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) + Map 6 + Map Operator Tree: + TableScan + alias: t2 + 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 + keys: c1 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 4 Data size: 381 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 4 Data size: 381 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 4 Data size: 419 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 = 0) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Statistics: Num rows: 2 Data size: 209 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 209 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 209 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 + 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 + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: SELECT c1 FROM t1 WHERE c1 NOT IN (SELECT c1 FROM t2) +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT c1 FROM t1 WHERE c1 NOT IN (SELECT c1 FROM t2) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +PREHOOK: query: explain SELECT c1 FROM t1 WHERE c1 NOT IN (SELECT c1 FROM t2 where t1.c2=t2.c1) +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT c1 FROM t1 WHERE c1 NOT IN (SELECT c1 FROM t2 where t1.c2=t2.c1) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (PARTITION-LEVEL SORT, 2), Reducer 13 (PARTITION-LEVEL SORT, 2) + Reducer 11 <- Reducer 10 (GROUP, 2) + Reducer 13 <- Map 12 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 12 (GROUP, 2) + Reducer 6 <- Map 8 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Reducer 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: c1 (type: int), c2 (type: char(100)) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + 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: 313 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 12 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: c2 (type: char(100)) + outputColumnNames: c2 + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: c2 (type: char(100)) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: char(100)) + sort order: + + Map-reduce partition columns: _col0 (type: char(100)) + Statistics: Num rows: 4 Data size: 313 Basic stats: COMPLETE Column stats: NONE + Map 8 + Map Operator Tree: + TableScan + alias: t2 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: c1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(_col0) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col0) (type: double) + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 9 + Map Operator Tree: + TableScan + alias: t2 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: c1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(_col0) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col0) (type: double) + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 10 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 UDFToDouble(_col0) (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int), _col1 (type: char(100)) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: char(100)) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: char(100)) + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: char(100)) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: char(100)), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: char(100)) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: char(100)) + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 13 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: char(100)) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 156 Basic stats: COMPLETE Column stats: NONE + 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: 156 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: char(100)) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: char(100)) + 1 _col0 (type: char(100)) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 4 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: char(100)) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: char(100)) + Statistics: Num rows: 4 Data size: 344 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int), _col1 (type: char(100)) + 1 _col0 (type: int), _col1 (type: char(100)) + outputColumnNames: _col0, _col3, _col4, _col7 + Statistics: Num rows: 4 Data size: 378 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 2 Data size: 189 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 189 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 189 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: char(100)) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 156 Basic stats: COMPLETE Column stats: NONE + 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: 156 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: char(100)) + Reducer 6 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 UDFToDouble(_col0) (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (type: char(100)) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: char(100)) + sort order: + + Map-reduce partition columns: _col0 (type: char(100)) + Statistics: Num rows: 3 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: char(100)) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: char(100)) + sort order: + + Map-reduce partition columns: _col0 (type: char(100)) + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT c1 FROM t1 WHERE c1 NOT IN (SELECT c1 FROM t2 where t1.c1=t2.c1) +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT c1 FROM t1 WHERE c1 NOT IN (SELECT c1 FROM t2 where t1.c1=t2.c1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +NULL +1 +PREHOOK: query: DROP TABLE t1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t1 +POSTHOOK: query: DROP TABLE t1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t1 +PREHOOK: query: DROP TABLE t2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t2 +POSTHOOK: query: DROP TABLE t2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t2 +PREHOOK: query: create table t1(a int, b int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: create table t1(a int, b int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +PREHOOK: query: insert into t1 values(1,0), (1,0),(1,0) +PREHOOK: type: QUERY +PREHOOK: Output: default@t1 +POSTHOOK: query: insert into t1 values(1,0), (1,0),(1,0) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@t1 +POSTHOOK: Lineage: t1.a EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: t1.b EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: create table t2(a int, b int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t2 +POSTHOOK: query: create table t2(a int, b int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t2 +PREHOOK: query: insert into t2 values(2,1), (3,1), (NULL,1) +PREHOOK: type: QUERY +PREHOOK: Output: default@t2 +POSTHOOK: query: insert into t2 values(2,1), (3,1), (NULL,1) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2.a EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: t2.b EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: explain select t1.a from t1 where t1.b NOT IN (select t2.a from t2 where t2.b=t1.a) +PREHOOK: type: QUERY +POSTHOOK: query: explain select t1.a from t1 where t1.b NOT IN (select t2.a from t2 where t2.b=t1.a) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: a (type: int), b (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 9 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: t2 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: b is not null (type: boolean) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(a) + keys: b (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: t2 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: b is not null (type: boolean) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: b (type: int), a (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 3 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 3 Data size: 9 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int), _col1 (type: int) + 1 _col1 (type: int), _col0 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4, _col7 + Statistics: Num rows: 3 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 6 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: int), _col0 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select t1.a from t1 where t1.b NOT IN (select t2.a from t2 where t2.b=t1.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: select t1.a from t1 where t1.b NOT IN (select t2.a from t2 where t2.b=t1.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +PREHOOK: query: drop table t1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t1 +POSTHOOK: query: drop table t1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t1 +PREHOOK: query: drop table t2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t2 +POSTHOOK: query: drop table t2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t2 +PREHOOK: query: create table t7(i int, j int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t7 +POSTHOOK: query: create table t7(i int, j int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t7 +PREHOOK: query: insert into t7 values(null, 5), (4, 15) +PREHOOK: type: QUERY +PREHOOK: Output: default@t7 +POSTHOOK: query: insert into t7 values(null, 5), (4, 15) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@t7 +POSTHOOK: Lineage: t7.i EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: t7.j EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: create table fixOb(i int, j int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@fixOb +POSTHOOK: query: create table fixOb(i int, j int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@fixOb +PREHOOK: query: insert into fixOb values(-1, 5), (-1, 15) +PREHOOK: type: QUERY +PREHOOK: Output: default@fixob +POSTHOOK: query: insert into fixOb values(-1, 5), (-1, 15) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@fixob +POSTHOOK: Lineage: fixob.i EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: fixob.j EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: explain select * from fixOb where j NOT IN (select i from t7 where t7.j=fixOb.j) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from fixOb where j NOT IN (select i from t7 where t7.j=fixOb.j) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 7 <- Reducer 6 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 9 <- Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: fixob + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: t7 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: j is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(i) + keys: j (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Map 5 + Map Operator Tree: + TableScan + alias: t7 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: j is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int), j (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Map 8 + Map Operator Tree: + TableScan + alias: fixob + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: j (type: int) + outputColumnNames: j + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: j (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + Left Outer Join 0 to 2 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + 2 _col3 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4, _col7 + Statistics: Num rows: 4 Data size: 19 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 9 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 9 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col2, _col3 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: int) + sort order: + + Map-reduce partition columns: _col3 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 9 + 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: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from fixOb where j NOT IN (select i from t7 where t7.j=fixOb.j) +PREHOOK: type: QUERY +PREHOOK: Input: default@fixob +PREHOOK: Input: default@t7 +#### A masked pattern was here #### +POSTHOOK: query: select * from fixOb where j NOT IN (select i from t7 where t7.j=fixOb.j) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@fixob +POSTHOOK: Input: default@t7 +#### A masked pattern was here #### +-1 15 +PREHOOK: query: drop table t7 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t7 +PREHOOK: Output: default@t7 +POSTHOOK: query: drop table t7 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t7 +POSTHOOK: Output: default@t7 +PREHOOK: query: drop table fixOb +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@fixob +PREHOOK: Output: default@fixob +POSTHOOK: query: drop table fixOb +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@fixob +POSTHOOK: Output: default@fixob +PREHOOK: query: create table t(i int, j int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t +POSTHOOK: query: create table t(i int, j int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t +PREHOOK: query: insert into t values(1,2), (4,5), (7, NULL) +PREHOOK: type: QUERY +PREHOOK: Output: default@t +POSTHOOK: query: insert into t values(1,2), (4,5), (7, NULL) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@t +POSTHOOK: Lineage: t.i EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: t.j EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: explain select t.i from t where t.j NOT IN (select t1.i from t t1 where t1.j=t.j) +PREHOOK: type: QUERY +POSTHOOK: query: explain select t.i from t where t.j NOT IN (select t1.i from t t1 where t1.j=t.j) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) + Reducer 6 <- Map 5 (GROUP, 2) + Reducer 7 <- Reducer 6 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 9 <- Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: j is not null (type: boolean) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(i) + keys: j (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Map 5 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: j is not null (type: boolean) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int), j (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Map 8 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: j (type: int) + outputColumnNames: j + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: j (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + Left Outer Join 0 to 2 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + 2 _col3 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4, _col7 + Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 3 Data size: 11 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 11 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 11 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col2, _col3 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: int) + sort order: + + Map-reduce partition columns: _col3 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select t.i from t where t.j NOT IN (select t1.i from t t1 where t1.j=t.j) +PREHOOK: type: QUERY +PREHOOK: Input: default@t +#### A masked pattern was here #### +POSTHOOK: query: select t.i from t where t.j NOT IN (select t1.i from t t1 where t1.j=t.j) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t +#### A masked pattern was here #### +4 +7 +1 +PREHOOK: query: explain select t.i from t where t.i NOT IN (select t1.i from t t1 where t1.j=t.j) +PREHOOK: type: QUERY +POSTHOOK: query: explain select t.i from t where t.i NOT IN (select t1.i from t t1 where t1.j=t.j) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: j is not null (type: boolean) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(i) + keys: j (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: j is not null (type: boolean) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int), j (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + 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, _col4 + Statistics: Num rows: 3 Data size: 11 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 3 Data size: 11 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int), _col1 (type: int) + 1 _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col3, _col4, _col7 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 8 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select t.i from t where t.i NOT IN (select t1.i from t t1 where t1.j=t.j) +PREHOOK: type: QUERY +PREHOOK: Input: default@t +#### A masked pattern was here #### +POSTHOOK: query: select t.i from t where t.i NOT IN (select t1.i from t t1 where t1.j=t.j) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t +#### A masked pattern was here #### +7 +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select t.i from t where t.j NOT IN (select t1.i from t t1 ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select t.i from t where t.j NOT IN (select t1.i from t t1 ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(i) + 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) + Map 6 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 61 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 3 Data size: 61 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: bigint), _col3 (type: bigint) + Reducer 3 + 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, _col2, _col3, _col5 + Statistics: Num rows: 3 Data size: 67 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = 0) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 1 Data size: 22 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 22 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 22 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 + 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 + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select t.i from t where t.j NOT IN (select t1.i from t t1 ) +PREHOOK: type: QUERY +PREHOOK: Input: default@t +#### A masked pattern was here #### +POSTHOOK: query: select t.i from t where t.j NOT IN (select t1.i from t t1 ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t +#### A masked pattern was here #### +4 +1 +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select t.i from t where t.i NOT IN (select t1.i from t t1 ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select t.i from t where t.i NOT IN (select t1.i from t t1 ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(i) + 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) + Map 6 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3 Data size: 61 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 61 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 3 Data size: 67 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 = 0) or (_col4 is null and _col0 is not null and (_col2 >= _col1))) (type: boolean) + Statistics: Num rows: 1 Data size: 22 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 22 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 22 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 + 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 + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select t.i from t where t.i NOT IN (select t1.i from t t1 ) +PREHOOK: type: QUERY +PREHOOK: Input: default@t +#### A masked pattern was here #### +POSTHOOK: query: select t.i from t where t.i NOT IN (select t1.i from t t1 ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t +#### A masked pattern was here #### +PREHOOK: query: drop table t1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table t1 +POSTHOOK: type: DROPTABLE +Warning: Shuffle Join JOIN[13][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 5' is a cross product +Warning: Shuffle Join JOIN[31][tables = [$hdt$_2, $hdt$_3]] in Work 'Reducer 10' is a cross product +PREHOOK: query: explain select * +from src b +where b.key not in + (select a.key + from src a + where b.value > a.value and a.key > '9' + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * +from src b +where b.key not in + (select a.key + from src a + where b.value > a.value and a.key > '9' + ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (PARTITION-LEVEL SORT, 1), Reducer 13 (PARTITION-LEVEL SORT, 1) + Reducer 11 <- Reducer 10 (GROUP, 2) + Reducer 13 <- Map 12 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Reducer 5 (GROUP, 2) + Reducer 8 <- Map 12 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + 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: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 12 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: value + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: value (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Map 4 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 9 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 10 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 41500 Data size: 923146 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col2 > _col1) (type: boolean) + Statistics: Num rows: 13833 Data size: 307707 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 13833 Data size: 307707 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col2 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13833 Data size: 307707 Basic stats: COMPLETE Column stats: NONE + 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: 13833 Data size: 307707 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6916 Data size: 153842 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 6916 Data size: 153842 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6916 Data size: 153842 Basic stats: COMPLETE Column stats: NONE + 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: 6916 Data size: 153842 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 13 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 7607 Data size: 169226 Basic stats: COMPLETE Column stats: NONE + 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: 7607 Data size: 169226 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col3, _col4, _col7 + Statistics: Num rows: 8367 Data size: 186148 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col0 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 4184 Data size: 93085 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4184 Data size: 93085 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 4184 Data size: 93085 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 41500 Data size: 923146 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col2 > _col1) (type: boolean) + Statistics: Num rows: 13833 Data size: 307707 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 13833 Data size: 307707 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + keys: _col2 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13833 Data size: 307707 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: 13833 Data size: 307707 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6916 Data size: 153842 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: 6916 Data size: 153842 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[13][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 5' is a cross product +Warning: Shuffle Join JOIN[31][tables = [$hdt$_2, $hdt$_3]] in Work 'Reducer 10' is a cross product +PREHOOK: query: select * +from src b +where b.key not in + (select a.key + from src a + where b.value > a.value and a.key > '9' + ) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from src b +where b.key not in + (select a.key + from src a + where b.value > a.value and a.key > '9' + ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +10 val_10 +103 val_103 +103 val_103 +105 val_105 +116 val_116 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +129 val_129 +129 val_129 +131 val_131 +134 val_134 +134 val_134 +136 val_136 +143 val_143 +145 val_145 +149 val_149 +149 val_149 +150 val_150 +155 val_155 +157 val_157 +158 val_158 +160 val_160 +163 val_163 +164 val_164 +164 val_164 +166 val_166 +17 val_17 +170 val_170 +172 val_172 +172 val_172 +180 val_180 +183 val_183 +189 val_189 +19 val_19 +190 val_190 +191 val_191 +191 val_191 +193 val_193 +193 val_193 +193 val_193 +195 val_195 +195 val_195 +196 val_196 +20 val_20 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +228 val_228 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +235 val_235 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +24 val_24 +24 val_24 +241 val_241 +244 val_244 +247 val_247 +248 val_248 +255 val_255 +255 val_255 +258 val_258 +26 val_26 +26 val_26 +260 val_260 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +28 val_28 +281 val_281 +281 val_281 +286 val_286 +291 val_291 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +30 val_30 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +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 +325 val_325 +325 val_325 +33 val_33 +331 val_331 +331 val_331 +332 val_332 +335 val_335 +339 val_339 +342 val_342 +342 val_342 +345 val_345 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +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 +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 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +41 val_41 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +42 val_42 +42 val_42 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +436 val_436 +44 val_44 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +459 val_459 +459 val_459 +462 val_462 +462 val_462 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +47 val_47 +472 val_472 +480 val_480 +480 val_480 +480 val_480 +482 val_482 +484 val_484 +485 val_485 +487 val_487 +490 val_490 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 +5 val_5 +5 val_5 +5 val_5 +51 val_51 +51 val_51 +54 val_54 +57 val_57 +58 val_58 +58 val_58 +65 val_65 +66 val_66 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +74 val_74 +77 val_77 +78 val_78 +8 val_8 +80 val_80 +84 val_84 +84 val_84 +85 val_85 +87 val_87 +9 val_9 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +100 val_100 +100 val_100 +104 val_104 +104 val_104 +11 val_11 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +133 val_133 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +146 val_146 +146 val_146 +15 val_15 +15 val_15 +152 val_152 +152 val_152 +153 val_153 +156 val_156 +162 val_162 +165 val_165 +165 val_165 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +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 +181 val_181 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +192 val_192 +194 val_194 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +2 val_2 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +208 val_208 +208 val_208 +208 val_208 +214 val_214 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +226 val_226 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +237 val_237 +237 val_237 +242 val_242 +242 val_242 +249 val_249 +252 val_252 +256 val_256 +256 val_256 +257 val_257 +262 val_262 +27 val_27 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +292 val_292 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +322 val_322 +322 val_322 +323 val_323 +327 val_327 +327 val_327 +327 val_327 +333 val_333 +333 val_333 +336 val_336 +338 val_338 +34 val_34 +341 val_341 +344 val_344 +344 val_344 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +35 val_35 +35 val_35 +35 val_35 +351 val_351 +362 val_362 +364 val_364 +365 val_365 +374 val_374 +392 val_392 +393 val_393 +397 val_397 +397 val_397 +4 val_4 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +43 val_43 +432 val_432 +435 val_435 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +460 val_460 +463 val_463 +463 val_463 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +481 val_481 +483 val_483 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +491 val_491 +492 val_492 +492 val_492 +53 val_53 +64 val_64 +67 val_67 +67 val_67 +72 val_72 +72 val_72 +76 val_76 +76 val_76 +82 val_82 +83 val_83 +83 val_83 +86 val_86 +90 val_90 +90 val_90 +90 val_90 +97 val_97 +97 val_97 +98 val_98 +98 val_98 diff --git a/ql/src/test/results/clientpositive/spark/subquery_null_agg.q.out b/ql/src/test/results/clientpositive/spark/subquery_null_agg.q.out new file mode 100644 index 0000000..144c3ec --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/subquery_null_agg.q.out @@ -0,0 +1,182 @@ +PREHOOK: query: CREATE TABLE table_7 (int_col INT) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table_7 +POSTHOOK: query: CREATE TABLE table_7 (int_col INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table_7 +Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[24][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain +SELECT +(t1.int_col) * (t1.int_col) AS int_col +FROM ( +SELECT +MIN(NULL) OVER () AS int_col +FROM table_7 +) t1 +WHERE +(False) NOT IN (SELECT +False AS boolean_col +FROM ( +SELECT +MIN(NULL) OVER () AS int_col +FROM table_7 +) tt1 +WHERE +(t1.int_col) = (tt1.int_col)) +PREHOOK: type: QUERY +POSTHOOK: query: explain +SELECT +(t1.int_col) * (t1.int_col) AS int_col +FROM ( +SELECT +MIN(NULL) OVER () AS int_col +FROM table_7 +) t1 +WHERE +(False) NOT IN (SELECT +False AS boolean_col +FROM ( +SELECT +MIN(NULL) OVER () AS int_col +FROM table_7 +) tt1 +WHERE +(t1.int_col) = (tt1.int_col)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: table_7 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: table_7 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + 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 + 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) + Map 6 + Map Operator Tree: + TableScan + alias: table_7 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + 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: false (type: boolean) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 9 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 = 0) or _col2 is null) (type: boolean) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: null (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 10 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 + 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 + 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: boolean) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: true (type: boolean) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + value expressions: _col0 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + diff --git a/ql/src/test/results/clientpositive/spark/subquery_scalar.q.out b/ql/src/test/results/clientpositive/spark/subquery_scalar.q.out new file mode 100644 index 0000000..470efca --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/subquery_scalar.q.out @@ -0,0 +1,6619 @@ +PREHOOK: query: create table tnull(i int, c char(2)) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tnull +POSTHOOK: query: create table tnull(i int, c char(2)) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tnull +PREHOOK: query: insert into tnull values(NULL, NULL), (NULL, NULL) +PREHOOK: type: QUERY +PREHOOK: Output: default@tnull +POSTHOOK: query: insert into tnull values(NULL, NULL), (NULL, NULL) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@tnull +POSTHOOK: Lineage: tnull.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: tnull.i EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +PREHOOK: query: create table tempty(c char(2)) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tempty +POSTHOOK: query: create table tempty(c char(2)) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tempty +PREHOOK: query: CREATE TABLE part_null( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +ROW FORMAT DELIMITED FIELDS TERMINATED BY "," +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@part_null +POSTHOOK: query: CREATE TABLE part_null( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +ROW FORMAT DELIMITED FIELDS TERMINATED BY "," +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@part_null +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny_nulls.txt' overwrite into table part_null +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@part_null +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny_nulls.txt' overwrite into table part_null +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@part_null +PREHOOK: query: insert into part_null values(78487,NULL,'Manufacturer#6','Brand#52','LARGE BRUSHED BRASS', 23, 'MED BAG',1464.48,'hely blith') +PREHOOK: type: QUERY +PREHOOK: Output: default@part_null +POSTHOOK: query: insert into part_null values(78487,NULL,'Manufacturer#6','Brand#52','LARGE BRUSHED BRASS', 23, 'MED BAG',1464.48,'hely blith') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part_null +POSTHOOK: Lineage: part_null.p_brand SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col4, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_comment SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col9, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_container SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col7, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_mfgr SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col3, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_name SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_partkey EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_retailprice EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col8, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_size EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col6, type:string, comment:), ] +POSTHOOK: Lineage: part_null.p_type SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col5, type:string, comment:), ] +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select * from part where p_size > (select avg(p_size) from part_null) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size > (select avg(p_size) from part_null) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 3 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: struct) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 26 Data size: 5149 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(_col5) > _col9) (type: boolean) + Statistics: Num rows: 8 Data size: 1584 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: 8 Data size: 1584 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 1584 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 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * from part where p_size > (select avg(p_size) from part_null) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size > (select avg(p_size) from part_null) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +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 +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +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 +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 +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 JOIN[15][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * from part where p_size > (select * from tempty) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@tempty +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size > (select * from tempty) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@tempty +#### A masked pattern was here #### +Warning: Shuffle Join JOIN[15][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select * from part where p_size > (select * from tempty) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size > (select * from tempty) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 3 + Map Operator Tree: + TableScan + alias: tempty + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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) + Map 5 + Map Operator Tree: + TableScan + alias: tempty + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: c (type: char(2)) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col0 (type: char(2)) + Reducer 2 + Reduce Operator Tree: + 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 + Statistics: Num rows: 26 Data size: 3381 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(_col5) > UDFToDouble(_col10)) (type: boolean) + Statistics: Num rows: 8 Data size: 1040 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: 8 Data size: 1040 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 1040 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 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select * from part where p_name = (select p_name from part_null where p_name is null) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_name = (select p_name from part_null where p_name is null) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 5 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_name = null) (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 13 Data size: 1573 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) + Map 3 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Map 5 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + 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 + Statistics: Num rows: 13 Data size: 44018 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) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 13 Data size: 44018 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 44018 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * from part where p_name = (select p_name from part_null where p_name is null) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_name = (select p_name from part_null where p_name is null) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[28][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from part where (select i from tnull limit 1) is null +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where (select i from tnull limit 1) is null +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 6 <- Reducer 5 (GROUP, 1) + Reducer 8 <- Map 7 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: tnull + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Map 7 + Map Operator Tree: + TableScan + alias: tnull + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3381 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3381 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 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 26 Data size: 3537 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col10 is null (type: boolean) + Statistics: Num rows: 13 Data size: 1768 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: 13 Data size: 1768 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1768 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 + 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 + 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 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[28][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from part where (select i from tnull limit 1) is null +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@tnull +#### A masked pattern was here #### +POSTHOOK: query: select * from part where (select i from tnull limit 1) is null +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@tnull +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 +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 JOIN[12][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select * from part where (select max(p_name) from part_null) is not null +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where (select max(p_name) from part_null) is not null +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 3 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_name) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 7957 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 7957 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 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not 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 + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * from part where (select max(p_name) from part_null) is not null +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part where (select max(p_name) from part_null) is not null +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 +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 JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from part where p_size between (select min(p_size) from part) and (select avg(p_size) from part) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size between (select min(p_size) from part) and (select avg(p_size) from part) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: struct) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3277 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), _col9 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + 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: 5279 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(_col5) BETWEEN UDFToDouble(_col9) AND _col10 (type: boolean) + Statistics: Num rows: 2 Data size: 406 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: 2 Data size: 406 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 406 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from part where p_size between (select min(p_size) from part) and (select avg(p_size) from part) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size between (select min(p_size) from part) and (select avg(p_size) from part) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +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 +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +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 JOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' 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) +PREHOOK: type: QUERY +POSTHOOK: 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) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (SORT, 1) + Reducer 6 <- Reducer 5 (GROUP, 1) + Reducer 8 <- Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 9 <- Reducer 8 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 + 1 + 2 + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 26 Data size: 6527 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col2 > _col4) (type: boolean) + Statistics: Num rows: 8 Data size: 2008 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col0 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 2008 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 2008 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 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: first_value_window_0 + arguments: _col5 + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: RANGE PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: first_value_window_0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 5 + Reduce Operator Tree: + Select Operator + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 121 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reducer 8 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col2, _col5 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string, _col5: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: first_value_window_0 + arguments: _col5 + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: RANGE PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: first_value_window_0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Reducer 9 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 121 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 121 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' 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) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: 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) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 +Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#2 almond antique violet turquoise frosted 40 +Manufacturer#2 almond aquamarine midnight light salmon 2 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 +Manufacturer#3 almond antique metallic orange dim 19 +Manufacturer#3 almond antique olive coral navajo 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#4 almond antique violet mint lemon 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 +Manufacturer#4 almond aquamarine yellow dodger mint 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#5 almond antique blue firebrick mint 31 +Manufacturer#5 almond antique medium spring khaki 6 +Manufacturer#5 almond antique sky peru orange 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 +Manufacturer#5 almond azure blanched chiffon midnight 23 +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select * from part where (p_partkey*p_size) <> (select min(p_partkey) from part) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where (p_partkey*p_size) <> (select min(p_partkey) from part) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int) + outputColumnNames: p_partkey + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_partkey) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col0 * _col5) <> _col9) (type: boolean) + Statistics: Num rows: 26 Data size: 3277 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: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3277 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * from part where (p_partkey*p_size) <> (select min(p_partkey) from part) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where (p_partkey*p_size) <> (select min(p_partkey) from part) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +PREHOOK: query: explain select count(*) as c from part as e where p_size + 100 < (select max(p_partkey) from part where p_name = e.p_name) +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(*) as c from part as e where p_size + 100 < (select max(p_partkey) from part where p_name = e.p_name) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 5 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: e + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_partkey) + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 + 100) < CASE WHEN (_col3 is null) THEN (null) ELSE (_col2) END) (type: boolean) + Statistics: Num rows: 9 Data size: 1112 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 9 Data size: 1112 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 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 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) as c from part as e where p_size + 100 < (select max(p_partkey) from part where p_name = e.p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select count(*) as c from part as e where p_size + 100 < (select max(p_partkey) from part where p_name = e.p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +26 +PREHOOK: query: explain select count(*) as c from part as e where 100 < (select max(p_partkey) from part where p_name = e.p_name) +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(*) as c from part as e where 100 < (select max(p_partkey) from part where p_name = e.p_name) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 5 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: e + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: _col0 + 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 + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_partkey) + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 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 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (100 < _col1) (type: boolean) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 484 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: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) as c from part as e where 100 < (select max(p_partkey) from part where p_name = e.p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select count(*) as c from part as e where 100 < (select max(p_partkey) from part where p_name = e.p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +26 +PREHOOK: query: explain select * from part where p_size > (select avg(p_size) from part_null where part_null.p_type = part.p_type) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size > (select avg(p_size) from part_null where part_null.p_type = part.p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 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) + Map 3 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: struct) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col4 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(_col5) > CASE WHEN (_col10 is null) THEN (null) ELSE (_col9) END) (type: boolean) + Statistics: Num rows: 9 Data size: 1112 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: 1112 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 9 Data size: 1112 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 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: double), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part where p_size > (select avg(p_size) from part_null where part_null.p_type = part.p_type) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size > (select avg(p_size) from part_null where part_null.p_type = part.p_type) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from part where p_size BETWEEN (select min(p_size) from part_null where part_null.p_type = part.p_type) AND (select max(p_size) from part_null) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size BETWEEN (select min(p_size) from part_null where part_null.p_type = part.p_type) AND (select max(p_size) from part_null) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 28 Data size: 3461 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), _col9 (type: int), _col10 (type: boolean) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col12 + Statistics: Num rows: 28 Data size: 3601 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col5 BETWEEN CASE WHEN (_col10 is null) THEN (null) ELSE (_col9) END AND _col12 (type: boolean) + Statistics: Num rows: 3 Data size: 385 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: 3 Data size: 385 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 385 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from part where p_size BETWEEN (select min(p_size) from part_null where part_null.p_type = part.p_type) AND (select max(p_size) from part_null) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size BETWEEN (select min(p_size) from part_null where part_null.p_type = part.p_type) AND (select max(p_size) from part_null) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 +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 JOIN[22][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from part where p_size >= (select min(p_size) from part_null where part_null.p_type = part.p_type) AND p_retailprice <= (select max(p_retailprice) from part_null) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size >= (select min(p_size) from part_null where part_null.p_type = part.p_type) AND p_retailprice <= (select max(p_retailprice) from part_null) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_retailprice (type: double) + outputColumnNames: p_retailprice + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_retailprice) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col4 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col5 >= CASE WHEN (_col10 is null) THEN (null) ELSE (_col9) END) (type: boolean) + Statistics: Num rows: 9 Data size: 1112 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: 1112 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 9 Data size: 1112 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 9 Data size: 1193 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col7 <= _col9) (type: boolean) + Statistics: Num rows: 3 Data size: 397 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: 3 Data size: 397 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 397 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from part where p_size >= (select min(p_size) from part_null where part_null.p_type = part.p_type) AND p_retailprice <= (select max(p_retailprice) from part_null) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size >= (select min(p_size) from part_null where part_null.p_type = part.p_type) AND p_retailprice <= (select max(p_retailprice) from part_null) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 +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 JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select * from part where p_brand <> (select min(p_brand) from part ) AND p_size IN (select (p_size) from part p where p.p_type = part.p_type ) AND p_size <> 340 +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_brand <> (select min(p_brand) from part ) AND p_size IN (select (p_size) from part p where p.p_type = part.p_type ) AND p_size <> 340 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_size <> 340) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string) + outputColumnNames: p_brand + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_brand) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string), p_size (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 26 Data size: 7957 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col3 <> _col9) (type: boolean) + Statistics: Num rows: 26 Data size: 7957 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: string), _col5 (type: int) + sort order: ++ + Map-reduce partition columns: _col4 (type: string), _col5 (type: int) + Statistics: Num rows: 26 Data size: 7957 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col4 (type: string), _col5 (type: int) + 1 _col1 (type: string), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 28 Data size: 8752 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 8752 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * from part where p_brand <> (select min(p_brand) from part ) AND p_size IN (select (p_size) from part p where p.p_type = part.p_type ) AND p_size <> 340 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_brand <> (select min(p_brand) from part ) AND p_size IN (select (p_size) from part p where p.p_type = part.p_type ) AND p_size <> 340 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +PREHOOK: query: explain select * from part where p_size <> (select count(p_name) from part p where p.p_size = part.p_size AND part.p_partkey= p.p_partkey ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size <> (select count(p_name) from part p where p.p_size = part.p_size AND part.p_partkey= p.p_partkey ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col5 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col5 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_partkey is not null and p_size is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(p_name) + keys: p_partkey (type: int), p_size (type: int) + 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: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int), _col5 (type: int) + 1 _col2 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToLong(_col5) <> CASE WHEN (_col10 is null) THEN (0) ELSE (_col9) END) (type: boolean) + Statistics: Num rows: 28 Data size: 3461 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: bigint), true (type: boolean), _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int), _col3 (type: int) + sort order: ++ + Map-reduce partition columns: _col2 (type: int), _col3 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part where p_size <> (select count(p_name) from part p where p.p_size = part.p_size AND part.p_partkey= p.p_partkey ) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size <> (select count(p_name) from part p where p.p_size = part.p_size AND part.p_partkey= p.p_partkey ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 +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 JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain select key, count(*) from src where value <> (select max(value) from src) group by key having count(*) > (select count(*) from src s1 where s1.key = '90' group by s1.key ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, count(*) from src where value <> (select max(value) from src) group by key having count(*) > (select count(*) from src s1 where s1.key = '90' group by s1.key ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP, 2) + Reducer 4 <- Reducer 11 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (GROUP, 1) + Reducer 8 <- Map 7 (GROUP, 2) + Reducer 9 <- Reducer 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + 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 + sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 10 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = '90') (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: '90' (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 5 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: value + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(value) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 7 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = '90') (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: '90' (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 500 Data size: 97812 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 <> _col2) (type: boolean) + Statistics: Num rows: 500 Data size: 97812 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 97812 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 97812 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: 97812 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 48906 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 250 Data size: 48906 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 + 1 + 2 + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 31250 Data size: 6726500 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 > _col3) (type: boolean) + Statistics: Num rows: 10416 Data size: 2242023 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10416 Data size: 2242023 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 10416 Data size: 2242023 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 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select key, count(*) from src where value <> (select max(value) from src) group by key having count(*) > (select count(*) from src s1 where s1.key = '90' group by s1.key ) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select key, count(*) from src where value <> (select max(value) from src) group by key having count(*) > (select count(*) from src s1 where s1.key = '90' group by s1.key ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +138 4 +169 4 +230 5 +277 4 +348 5 +401 5 +406 4 +468 4 +469 5 +489 4 +Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select sum(p_retailprice) from part group by p_type having sum(p_retailprice) > (select max(pp.p_retailprice) from part pp) +PREHOOK: type: QUERY +POSTHOOK: query: explain select sum(p_retailprice) from part group by p_type having sum(p_retailprice) > (select max(pp.p_retailprice) from part pp) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_retailprice (type: double) + outputColumnNames: p_type, p_retailprice + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(p_retailprice) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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) + Map 4 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_retailprice (type: double) + outputColumnNames: p_retailprice + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_retailprice) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: double) + outputColumnNames: _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: double) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 13 Data size: 1690 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 > _col2) (type: boolean) + Statistics: Num rows: 4 Data size: 520 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 520 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 520 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 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select sum(p_retailprice) from part group by p_type having sum(p_retailprice) > (select max(pp.p_retailprice) from part pp) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select sum(p_retailprice) from part group by p_type having sum(p_retailprice) > (select max(pp.p_retailprice) from part pp) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +2346.3 +3461.37 +Warning: Shuffle Join JOIN[72][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' 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 +POSTHOOK: query: explain select * from part where p_size > (select count(p_name) from part INTERSECT select count(p_brand) from part) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Reducer 16 (GROUP, 2) + Reducer 12 <- Map 11 (GROUP, 1) + Reducer 13 <- Reducer 12 (GROUP, 2) + Reducer 14 <- Reducer 10 (GROUP, 2), Reducer 13 (GROUP, 2) + Reducer 16 <- Map 15 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 14 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Reducer 10 (GROUP, 2), Reducer 13 (GROUP, 2) + Reducer 7 <- Reducer 6 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 11 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(p_name) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Map 15 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_brand (type: string) + outputColumnNames: p_brand + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(p_brand) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 10 + 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: 8 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col1) + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 12 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 13 + 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: 8 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col1) + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 14 + 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: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 = 2) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 16 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + 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 + Statistics: Num rows: 26 Data size: 3797 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToLong(_col5) > _col10) (type: boolean) + Statistics: Num rows: 8 Data size: 1168 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: 8 Data size: 1168 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 1168 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 + 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: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 = 2) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[72][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' 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 +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size > (select count(p_name) from part INTERSECT select count(p_brand) from part) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +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 +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 +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 +PREHOOK: query: explain select p_partkey from part where p_name like (select max(p.p_name) from part p left outer join part pp on p.p_type = pp.p_type where pp.p_size = part.p_size) +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_partkey from part where p_name like (select max(p.p_name) from part p left outer join part pp on p.p_type = pp.p_type where pp.p_size = part.p_size) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_type (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col2 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 like CASE WHEN (_col4 is null) THEN (null) ELSE (_col3) END) (type: boolean) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 14 Data size: 1730 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + keys: _col3 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select p_partkey from part where p_name like (select max(p.p_name) from part p left outer join part pp on p.p_type = pp.p_type where pp.p_size = part.p_size) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_partkey from part where p_name like (select max(p.p_name) from part p left outer join part pp on p.p_type = pp.p_type where pp.p_size = part.p_size) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +105685 +110592 +112398 +132666 +144293 +146985 +15103 +17927 +191709 +195606 +33357 +40982 +45261 +48427 +49671 +65667 +78486 +85768 +86428 +90681 +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from part_null where p_name NOT LIKE (select min(p_name) from part_null) AND p_brand NOT IN (select p_name from part) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name NOT LIKE (select min(p_name) from part_null) AND p_brand NOT IN (select p_name from part) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 1) + Reducer 8 <- Map 7 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3256 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) + Map 5 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_name) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 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) + Map 9 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 10 + 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: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 1 Data size: 3441 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not (_col1 like _col9)) (type: boolean) + Statistics: Num rows: 1 Data size: 3441 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: 3441 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3441 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 + Reduce Operator Tree: + 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: 3458 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: 3458 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 + Reduce Operator Tree: + 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: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col10 = 0) or (_col13 is null and _col3 is not null and (_col11 >= _col10))) (type: boolean) + Statistics: Num rows: 9 Data size: 1112 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: 1112 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 9 Data size: 1112 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 8 + 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 + 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) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from part_null where p_name NOT LIKE (select min(p_name) from part_null) AND p_brand NOT IN (select p_name from part) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name NOT LIKE (select min(p_name) from part_null) AND p_brand NOT IN (select p_name from part) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +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 +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 +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 JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select * from part_null where p_brand NOT IN (select p_name from part) AND p_name NOT LIKE (select min(p_name) from part_null pp where part_null.p_type = pp.p_type) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_brand NOT IN (select p_name from part) AND p_name NOT LIKE (select min(p_name) from part_null pp where part_null.p_type = pp.p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 1) + Reducer 8 <- Map 7 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3256 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) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 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) + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_name (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Map 9 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_name) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 10 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3256 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: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner 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: 3273 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: 3273 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 + Reduce Operator Tree: + 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, _col9, _col10, _col12 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not CASE WHEN ((_col9 = 0)) THEN (false) WHEN (_col12 is not null) THEN (true) WHEN (_col3 is null) THEN (null) WHEN ((_col10 < _col9)) THEN (true) ELSE (false) END) (type: boolean) + Statistics: Num rows: 7 Data size: 865 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: 7 Data size: 865 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: 7 Data size: 865 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 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col13, _col14 + Statistics: Num rows: 7 Data size: 951 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not (_col1 like CASE WHEN (_col14 is null) THEN (null) ELSE (_col13) END)) (type: boolean) + Statistics: Num rows: 4 Data size: 543 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: 543 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 543 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 + 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 + 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) + Reducer 8 + 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: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select * from part_null where p_brand NOT IN (select p_name from part) AND p_name NOT LIKE (select min(p_name) from part_null pp where part_null.p_type = pp.p_type) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_brand NOT IN (select p_name from part) AND p_name NOT LIKE (select min(p_name) from part_null pp where part_null.p_type = pp.p_type) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### 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 +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 +li.l_orderkey <> (select min(l_orderkey) from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +PREHOOK: type: QUERY +POSTHOOK: 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 +li.l_orderkey <> (select min(l_orderkey) from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: li + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((l_linenumber = 1) and l_partkey is not null) (type: boolean) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((l_shipmode = 'AIR') and l_linenumber is not null) (type: boolean) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), l_linenumber (type: int) + outputColumnNames: l_orderkey, l_linenumber + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(l_orderkey) + keys: l_linenumber (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col1 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 55 Data size: 6598 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: int) + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 55 Data size: 6598 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int), _col3 (type: int) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col4 (type: int) + 1 _col2 (type: int) + outputColumnNames: _col0, _col1, _col3, _col5, _col6 + Statistics: Num rows: 60 Data size: 7257 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 <> CASE WHEN (_col6 is null) THEN (null) ELSE (_col5) END) (type: boolean) + Statistics: Num rows: 60 Data size: 7257 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 60 Data size: 7257 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 60 Data size: 7257 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 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +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 +li.l_orderkey <> (select min(l_orderkey) from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem +#### A masked pattern was here #### +POSTHOOK: 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 +li.l_orderkey <> (select min(l_orderkey) from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem +#### A masked pattern was here #### +106170 1191 +108570 8571 +115118 7630 +115209 7721 +119477 1989 +119767 9768 +123076 613 +139636 2150 +155190 7706 +175839 874 +182052 9607 +21636 9143 +22630 5133 +2320 9821 +40216 217 +450 2951 +59694 4705 +61336 8855 +61931 1932 +64128 9141 +7068 9569 +82704 7721 +85951 5952 +88035 5560 +88362 871 +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 + li.l_orderkey <> (select min(l_orderkey) from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +PREHOOK: type: QUERY +POSTHOOK: 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 + li.l_orderkey <> (select min(l_orderkey) from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: li + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((l_linenumber = 1) and l_partkey is not null) (type: boolean) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((l_shipmode = 'AIR') and l_linenumber is not null) (type: boolean) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), l_linenumber (type: int) + outputColumnNames: l_orderkey, l_linenumber + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(l_orderkey) + keys: l_linenumber (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col1 (type: int) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 55 Data size: 6598 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: int) + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 55 Data size: 6598 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int), _col3 (type: int) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col4 (type: int) + 1 _col2 (type: int) + outputColumnNames: _col0, _col1, _col3, _col5, _col6 + Statistics: Num rows: 60 Data size: 7257 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 <> CASE WHEN (_col6 is null) THEN (null) ELSE (_col5) END) (type: boolean) + Statistics: Num rows: 60 Data size: 7257 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 60 Data size: 7257 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 60 Data size: 7257 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 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +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 +li.l_orderkey <> (select min(l_orderkey) from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem +#### A masked pattern was here #### +POSTHOOK: 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 +li.l_orderkey <> (select min(l_orderkey) from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem +#### A masked pattern was here #### +106170 1191 +108570 8571 +115118 7630 +115209 7721 +119477 1989 +119767 9768 +123076 613 +139636 2150 +155190 7706 +175839 874 +182052 9607 +21636 9143 +22630 5133 +2320 9821 +40216 217 +450 2951 +59694 4705 +61336 8855 +61931 1932 +64128 9141 +7068 9569 +82704 7721 +85951 5952 +88035 5560 +88362 871 +PREHOOK: query: explain select sum(l_extendedprice) from lineitem, part where p_partkey = l_partkey and l_quantity > (select avg(l_quantity) from lineitem where l_partkey = p_partkey) +PREHOOK: type: QUERY +POSTHOOK: query: explain select sum(l_extendedprice) from lineitem, part where p_partkey = l_partkey and l_quantity > (select avg(l_quantity) from lineitem where l_partkey = p_partkey) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 6 <- Map 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_partkey (type: int), l_quantity (type: double), l_extendedprice (type: double) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: double), _col2 (type: double) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(l_quantity) + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: struct) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + 2 _col1 (type: int) + outputColumnNames: _col1, _col2, _col4 + Statistics: Num rows: 220 Data size: 26397 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 > _col4) (type: boolean) + Statistics: Num rows: 73 Data size: 8759 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: double) + outputColumnNames: _col2 + Statistics: Num rows: 73 Data size: 8759 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_col2) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 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 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: double), _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select sum(l_extendedprice) from lineitem, part where p_partkey = l_partkey and l_quantity > (select avg(l_quantity) from lineitem where l_partkey = p_partkey) +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select sum(l_extendedprice) from lineitem, part where p_partkey = l_partkey and l_quantity > (select avg(l_quantity) from lineitem where l_partkey = p_partkey) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem +POSTHOOK: Input: default@part +#### A masked pattern was here #### +NULL +PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type AND p_brand NOT LIKE (select min(p_brand) from part pp where part.p_type = pp.p_type)) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type AND p_brand NOT LIKE (select min(p_brand) from part pp where part.p_type = pp.p_type)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part_null + Statistics: Num rows: 1 Data size: 3256 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col4 (type: string) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col4 (type: string) + Statistics: Num rows: 1 Data size: 3256 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_brand (type: string), p_type (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_brand) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string), _col4 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 1903 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 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not (_col1 like CASE WHEN (_col4 is null) THEN (null) ELSE (_col3) END)) (type: boolean) + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + 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: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type AND p_brand NOT LIKE (select min(p_brand) from part pp where part.p_type = pp.p_type)) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@part_null +#### A masked pattern was here #### +POSTHOOK: query: select * from part_null where p_name IN (select p_name from part where part.p_type = part_null.p_type AND p_brand NOT LIKE (select min(p_brand) from part pp where part.p_type = pp.p_type)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@part_null +#### A masked pattern was here #### +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +PREHOOK: query: drop table tnull +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tnull +PREHOOK: Output: default@tnull +POSTHOOK: query: drop table tnull +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tnull +POSTHOOK: Output: default@tnull +PREHOOK: query: drop table part_null +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@part_null +PREHOOK: Output: default@part_null +POSTHOOK: query: drop table part_null +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@part_null +POSTHOOK: Output: default@part_null +PREHOOK: query: drop table tempty +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tempty +PREHOOK: Output: default@tempty +POSTHOOK: query: drop table tempty +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tempty +POSTHOOK: Output: default@tempty +PREHOOK: query: create table EMPS(EMPNO int,NAME string,DEPTNO int,GENDER string,CITY string,EMPID int,AGE int,SLACKER boolean,MANAGER boolean,JOINEDAT date) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@EMPS +POSTHOOK: query: create table EMPS(EMPNO int,NAME string,DEPTNO int,GENDER string,CITY string,EMPID int,AGE int,SLACKER boolean,MANAGER boolean,JOINEDAT date) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@EMPS +PREHOOK: query: insert into EMPS values (100,'Fred',10,NULL,NULL,30,25,true,false,'1996-08-03') +PREHOOK: type: QUERY +PREHOOK: Output: default@emps +POSTHOOK: query: insert into EMPS values (100,'Fred',10,NULL,NULL,30,25,true,false,'1996-08-03') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@emps +POSTHOOK: Lineage: emps.age EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col7, type:string, comment:), ] +POSTHOOK: Lineage: emps.city SIMPLE [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col5, type:string, comment:), ] +POSTHOOK: Lineage: emps.deptno EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col3, type:string, comment:), ] +POSTHOOK: Lineage: emps.empid EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col6, type:string, comment:), ] +POSTHOOK: Lineage: emps.empno EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: emps.gender SIMPLE [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col4, type:string, comment:), ] +POSTHOOK: Lineage: emps.joinedat EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col10, type:string, comment:), ] +POSTHOOK: Lineage: emps.manager EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col9, type:string, comment:), ] +POSTHOOK: Lineage: emps.name SIMPLE [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: emps.slacker EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col8, type:string, comment:), ] +PREHOOK: query: insert into EMPS values (110,'Eric',20,'M','San Francisco',3,80,NULL,false,'2001-01-01') +PREHOOK: type: QUERY +PREHOOK: Output: default@emps +POSTHOOK: query: insert into EMPS values (110,'Eric',20,'M','San Francisco',3,80,NULL,false,'2001-01-01') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@emps +POSTHOOK: Lineage: emps.age EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col7, type:string, comment:), ] +POSTHOOK: Lineage: emps.city SIMPLE [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col5, type:string, comment:), ] +POSTHOOK: Lineage: emps.deptno EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col3, type:string, comment:), ] +POSTHOOK: Lineage: emps.empid EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col6, type:string, comment:), ] +POSTHOOK: Lineage: emps.empno EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: emps.gender SIMPLE [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col4, type:string, comment:), ] +POSTHOOK: Lineage: emps.joinedat EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col10, type:string, comment:), ] +POSTHOOK: Lineage: emps.manager EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col9, type:string, comment:), ] +POSTHOOK: Lineage: emps.name SIMPLE [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: emps.slacker EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col8, type:string, comment:), ] +PREHOOK: query: insert into EMPS values (110,'John',40,'M','Vancouver',2,NULL,false,true,'2002-05-03') +PREHOOK: type: QUERY +PREHOOK: Output: default@emps +POSTHOOK: query: insert into EMPS values (110,'John',40,'M','Vancouver',2,NULL,false,true,'2002-05-03') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@emps +POSTHOOK: Lineage: emps.age EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col7, type:string, comment:), ] +POSTHOOK: Lineage: emps.city SIMPLE [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col5, type:string, comment:), ] +POSTHOOK: Lineage: emps.deptno EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col3, type:string, comment:), ] +POSTHOOK: Lineage: emps.empid EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col6, type:string, comment:), ] +POSTHOOK: Lineage: emps.empno EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: emps.gender SIMPLE [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col4, type:string, comment:), ] +POSTHOOK: Lineage: emps.joinedat EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col10, type:string, comment:), ] +POSTHOOK: Lineage: emps.manager EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col9, type:string, comment:), ] +POSTHOOK: Lineage: emps.name SIMPLE [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: emps.slacker EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col8, type:string, comment:), ] +PREHOOK: query: insert into EMPS values (120,'Wilma',20,'F',NULL,1,5,NULL,true,'2005-09-07') +PREHOOK: type: QUERY +PREHOOK: Output: default@emps +POSTHOOK: query: insert into EMPS values (120,'Wilma',20,'F',NULL,1,5,NULL,true,'2005-09-07') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@emps +POSTHOOK: Lineage: emps.age EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col7, type:string, comment:), ] +POSTHOOK: Lineage: emps.city SIMPLE [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col5, type:string, comment:), ] +POSTHOOK: Lineage: emps.deptno EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col3, type:string, comment:), ] +POSTHOOK: Lineage: emps.empid EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col6, type:string, comment:), ] +POSTHOOK: Lineage: emps.empno EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: emps.gender SIMPLE [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col4, type:string, comment:), ] +POSTHOOK: Lineage: emps.joinedat EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col10, type:string, comment:), ] +POSTHOOK: Lineage: emps.manager EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col9, type:string, comment:), ] +POSTHOOK: Lineage: emps.name SIMPLE [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: emps.slacker EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col8, type:string, comment:), ] +PREHOOK: query: insert into EMPS values (130,'Alice',40,'F','Vancouver',2,NULL,false,true,'2007-01-01') +PREHOOK: type: QUERY +PREHOOK: Output: default@emps +POSTHOOK: query: insert into EMPS values (130,'Alice',40,'F','Vancouver',2,NULL,false,true,'2007-01-01') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@emps +POSTHOOK: Lineage: emps.age EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col7, type:string, comment:), ] +POSTHOOK: Lineage: emps.city SIMPLE [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col5, type:string, comment:), ] +POSTHOOK: Lineage: emps.deptno EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col3, type:string, comment:), ] +POSTHOOK: Lineage: emps.empid EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col6, type:string, comment:), ] +POSTHOOK: Lineage: emps.empno EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: emps.gender SIMPLE [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col4, type:string, comment:), ] +POSTHOOK: Lineage: emps.joinedat EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col10, type:string, comment:), ] +POSTHOOK: Lineage: emps.manager EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col9, type:string, comment:), ] +POSTHOOK: Lineage: emps.name SIMPLE [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +POSTHOOK: Lineage: emps.slacker EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col8, type:string, comment:), ] +PREHOOK: query: create table DEPTS(deptno int, name string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@DEPTS +POSTHOOK: query: create table DEPTS(deptno int, name string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@DEPTS +PREHOOK: query: insert into DEPTS values( 10,'Sales') +PREHOOK: type: QUERY +PREHOOK: Output: default@depts +POSTHOOK: query: insert into DEPTS values( 10,'Sales') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@depts +POSTHOOK: Lineage: depts.deptno EXPRESSION [(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: depts.name SIMPLE [(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: insert into DEPTS values( 20,'Marketing') +PREHOOK: type: QUERY +PREHOOK: Output: default@depts +POSTHOOK: query: insert into DEPTS values( 20,'Marketing') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@depts +POSTHOOK: Lineage: depts.deptno EXPRESSION [(values__tmp__table__9)values__tmp__table__9.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: depts.name SIMPLE [(values__tmp__table__9)values__tmp__table__9.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: insert into DEPTS values( 30,'Accounts') +PREHOOK: type: QUERY +PREHOOK: Output: default@depts +POSTHOOK: query: insert into DEPTS values( 30,'Accounts') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@depts +POSTHOOK: Lineage: depts.deptno EXPRESSION [(values__tmp__table__10)values__tmp__table__10.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: depts.name SIMPLE [(values__tmp__table__10)values__tmp__table__10.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: explain select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: emps + Statistics: Num rows: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: empno (type: int), name (type: string), deptno (type: int), gender (type: string), city (type: string), empid (type: int), age (type: int), slacker (type: boolean), manager (type: boolean), joinedat (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 5 Data size: 238 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: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + Map 3 + Map Operator Tree: + TableScan + alias: depts + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: name is not null (type: boolean) + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(deptno) + keys: name (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 31 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: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToLong(_col2) <> CASE WHEN (_col11 is null) THEN (0) ELSE (_col10) END) (type: boolean) + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 261 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 10 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: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) +PREHOOK: type: QUERY +PREHOOK: Input: default@depts +PREHOOK: Input: default@emps +#### A masked pattern was here #### +POSTHOOK: query: select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@depts +POSTHOOK: Input: default@emps +#### A masked pattern was here #### +100 Fred 10 NULL NULL 30 25 true false 1996-08-03 +110 Eric 20 M San Francisco 3 80 NULL false 2001-01-01 +110 John 40 M Vancouver 2 NULL false true 2002-05-03 +120 Wilma 20 F NULL 1 5 NULL true 2005-09-07 +130 Alice 40 F Vancouver 2 NULL false true 2007-01-01 +PREHOOK: query: explain select * from emps where name > (select min(name) from depts where depts.deptno=emps.deptno) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from emps where name > (select min(name) from depts where depts.deptno=emps.deptno) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: emps + Statistics: Num rows: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: empno (type: int), name (type: string), deptno (type: int), gender (type: string), city (type: string), empid (type: int), age (type: int), slacker (type: boolean), manager (type: boolean), joinedat (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + Map 3 + Map Operator Tree: + TableScan + alias: depts + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: deptno is not null (type: boolean) + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(name) + keys: deptno (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 > CASE WHEN (_col11 is null) THEN (null) ELSE (_col10) END) (type: boolean) + Statistics: Num rows: 1 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 1 Data size: 52 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 52 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from emps where name > (select min(name) from depts where depts.deptno=emps.deptno) +PREHOOK: type: QUERY +PREHOOK: Input: default@depts +PREHOOK: Input: default@emps +#### A masked pattern was here #### +POSTHOOK: query: select * from emps where name > (select min(name) from depts where depts.deptno=emps.deptno) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@depts +POSTHOOK: Input: default@emps +#### A masked pattern was here #### +120 Wilma 20 F NULL 1 5 NULL true 2005-09-07 +PREHOOK: query: explain select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) and empno > (select count(name) from depts where depts.deptno = emps.deptno) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) and empno > (select count(name) from depts where depts.deptno = emps.deptno) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: emps + Statistics: Num rows: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: empno (type: int), name (type: string), deptno (type: int), gender (type: string), city (type: string), empid (type: int), age (type: int), slacker (type: boolean), manager (type: boolean), joinedat (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 5 Data size: 238 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: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + Map 4 + Map Operator Tree: + TableScan + alias: depts + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: name is not null (type: boolean) + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(deptno) + keys: name (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 31 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: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: depts + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: deptno is not null (type: boolean) + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(name) + keys: deptno (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToLong(_col2) <> CASE WHEN (_col11 is null) THEN (0) ELSE (_col10) END) (type: boolean) + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col13, _col14 + Statistics: Num rows: 5 Data size: 287 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToLong(_col0) > CASE WHEN (_col14 is null) THEN (0) ELSE (_col13) END) (type: boolean) + Statistics: Num rows: 1 Data size: 57 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 1 Data size: 57 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 57 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 10 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: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) and empno > (select count(name) from depts where depts.deptno = emps.deptno) +PREHOOK: type: QUERY +PREHOOK: Input: default@depts +PREHOOK: Input: default@emps +#### A masked pattern was here #### +POSTHOOK: query: select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) and empno > (select count(name) from depts where depts.deptno = emps.deptno) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@depts +POSTHOOK: Input: default@emps +#### A masked pattern was here #### +100 Fred 10 NULL NULL 30 25 true false 1996-08-03 +110 Eric 20 M San Francisco 3 80 NULL false 2001-01-01 +110 John 40 M Vancouver 2 NULL false true 2002-05-03 +120 Wilma 20 F NULL 1 5 NULL true 2005-09-07 +130 Alice 40 F Vancouver 2 NULL false true 2007-01-01 +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select * from emps where deptno <> (select sum(deptno) from depts where depts.name = emps.name) and empno > (select count(name) from depts) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from emps where deptno <> (select sum(deptno) from depts where depts.name = emps.name) and empno > (select count(name) from depts) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: emps + Statistics: Num rows: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: empno (type: int), name (type: string), deptno (type: int), gender (type: string), city (type: string), empid (type: int), age (type: int), slacker (type: boolean), manager (type: boolean), joinedat (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 5 Data size: 238 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: 5 Data size: 238 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + Map 4 + Map Operator Tree: + TableScan + alias: depts + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: name is not null (type: boolean) + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(deptno) + keys: name (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 31 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: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: depts + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: name (type: string) + outputColumnNames: name + Statistics: Num rows: 3 Data size: 31 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(name) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToLong(_col2) <> CASE WHEN (_col11 is null) THEN (null) ELSE (_col10) END) (type: boolean) + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 5 Data size: 261 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 5 Data size: 306 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToLong(_col0) > _col10) (type: boolean) + Statistics: Num rows: 1 Data size: 61 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: int), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 1 Data size: 61 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 61 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 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 10 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: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) and empno > (select count(name) from depts) +PREHOOK: type: QUERY +PREHOOK: Input: default@depts +PREHOOK: Input: default@emps +#### A masked pattern was here #### +POSTHOOK: query: select * from emps where deptno <> (select count(deptno) from depts where depts.name = emps.name) and empno > (select count(name) from depts) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@depts +POSTHOOK: Input: default@emps +#### A masked pattern was here #### +100 Fred 10 NULL NULL 30 25 true false 1996-08-03 +110 Eric 20 M San Francisco 3 80 NULL false 2001-01-01 +110 John 40 M Vancouver 2 NULL false true 2002-05-03 +120 Wilma 20 F NULL 1 5 NULL true 2005-09-07 +130 Alice 40 F Vancouver 2 NULL false true 2007-01-01 +PREHOOK: query: drop table DEPTS +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@depts +PREHOOK: Output: default@depts +POSTHOOK: query: drop table DEPTS +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@depts +POSTHOOK: Output: default@depts +PREHOOK: query: drop table EMPS +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@emps +PREHOOK: Output: default@emps +POSTHOOK: query: drop table EMPS +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@emps +POSTHOOK: Output: default@emps +Warning: Shuffle Join JOIN[15][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain + select key, count(*) +from src +group by key +having count(*) > (select count(*) from src s1 where s1.key > '9' ) +PREHOOK: type: QUERY +POSTHOOK: query: explain + select key, count(*) +from src +group by key +having count(*) > (select count(*) from src s1 where s1.key > '9' ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + 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) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: key (type: string) + mode: hash + 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: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 4906 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 > _col2) (type: boolean) + Statistics: Num rows: 83 Data size: 1628 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 1628 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 83 Data size: 1628 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[15][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select key, count(*) +from src +group by key +having count(*) > (select count(*) from src s1 where s1.key = '90') +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select key, count(*) +from src +group by key +having count(*) > (select count(*) from src s1 where s1.key = '90') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +138 4 +169 4 +230 5 +277 4 +348 5 +401 5 +406 4 +468 4 +469 5 +489 4 +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) > (select count(*) from src s1 where s1.key > '9' ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) > (select count(*) from src s1 where s1.key > '9' ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '8') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '8') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE + 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: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 91 Data size: 1788 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col2 > _col3) (type: boolean) + Statistics: Num rows: 30 Data size: 589 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 30 Data size: 589 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 30 Data size: 589 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 7 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 4' 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') +group by key, value +having count(*) > (select count(*) from src s1 where s1.key > '9' ) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +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 JOIN[24][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' 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 +POSTHOOK: query: explain select * from part where p_size > (select max(p_size) from part group by p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 2) + Reducer 5 <- Reducer 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 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) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string) + outputColumnNames: p_type + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: p_type, p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + 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 + Statistics: Num rows: 338 Data size: 84851 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col5 > _col10) (type: boolean) + Statistics: Num rows: 112 Data size: 28116 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: 112 Data size: 28116 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 112 Data size: 28116 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 + 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: NONE + Select Operator + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from part where p_size > (select max(p_size) from part p where p.p_type = part.p_type group by p_type) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size > (select max(p_size) from part p where p.p_type = part.p_type group by p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 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) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Map 5 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 _col4 (type: string) + 1 _col0 (type: string) + 2 _col1 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col11 + residual filter predicates: {(_col5 > _col11)} + Statistics: Num rows: 57 Data size: 6923 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: 57 Data size: 6923 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 57 Data size: 6923 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 + 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: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col1) <= 1) (type: boolean) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 242 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: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Reducer 6 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain select * from part where p_size <> + (select count(p_size) from part pp where part.p_type <> pp.p_type) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from part where p_size <> + (select count(p_size) from part pp where part.p_type <> pp.p_type) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 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 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 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) + Map 3 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string) + outputColumnNames: p_type + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 185 Data size: 45180 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToLong(_col5) <> CASE WHEN (_col10 is null) THEN (0) ELSE (_col9) END) (type: boolean) + Statistics: Num rows: 185 Data size: 45180 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: 185 Data size: 45180 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 185 Data size: 45180 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 338 Data size: 82147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col2 <> _col0) (type: boolean) + Statistics: Num rows: 338 Data size: 82147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col2 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 338 Data size: 82147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col1) + keys: _col2 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 338 Data size: 82147 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: 338 Data size: 82147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 169 Data size: 41073 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 169 Data size: 41073 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: 169 Data size: 41073 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + Reducer 7 + 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: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select * from part where p_size <> + (select count(p_size) from part pp where part.p_type <> pp.p_type) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from part where p_size <> + (select count(p_size) from part pp where part.p_type <> pp.p_type) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +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 +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +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 +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +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 +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +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 +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +PREHOOK: query: create table t(i int, j int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t +POSTHOOK: query: create table t(i int, j int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t +PREHOOK: query: insert into t values(3,1), (1,1) +PREHOOK: type: QUERY +PREHOOK: Output: default@t +POSTHOOK: query: insert into t values(3,1), (1,1) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@t +POSTHOOK: Lineage: t.i EXPRESSION [(values__tmp__table__11)values__tmp__table__11.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: t.j EXPRESSION [(values__tmp__table__11)values__tmp__table__11.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain select * from t where 0 = (select count(*) from t tt where tt.j <> t.i) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from t where 0 = (select count(*) from t tt where tt.j <> t.i) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: tt + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: j (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: CASE WHEN (_col3 is null) THEN ((0 = 0)) ELSE ((_col2 = 0)) END (type: boolean) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col0 <> _col1) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col1 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select * from t where 0 = (select count(*) from t tt where tt.j <> t.i) +PREHOOK: type: QUERY +PREHOOK: Input: default@t +#### A masked pattern was here #### +POSTHOOK: query: select * from t where 0 = (select count(*) from t tt where tt.j <> t.i) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t +#### A masked pattern was here #### +1 1 +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain select * from t where 0 = (select avg(tt.i) from t tt where tt.j <> t.i) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from t where 0 = (select avg(tt.i) from t tt where tt.j <> t.i) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: tt + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 6 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 <> _col2) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col2 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(_col0) + keys: _col2 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: struct) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (0.0 = _col1) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select * from t where 0 = (select avg(tt.i) from t tt where tt.j <> t.i) +PREHOOK: type: QUERY +PREHOOK: Input: default@t +#### A masked pattern was here #### +POSTHOOK: query: select * from t where 0 = (select avg(tt.i) from t tt where tt.j <> t.i) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t +#### A masked pattern was here #### +PREHOOK: query: create table tempty(i int, j int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tempty +POSTHOOK: query: create table tempty(i int, j int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tempty +PREHOOK: query: explain select * from t where 0 = (select count(*) from tempty tt where t.i=tt.i) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from t where 0 = (select count(*) from tempty tt where t.i=tt.i) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: tt + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: i is not null (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count() + keys: i (type: int) + mode: hash + outputColumnNames: _col0, _col1 + 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: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: CASE WHEN (_col3 is null) THEN ((0 = 0)) ELSE ((_col2 = 0)) END (type: boolean) + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from t where 0 = (select count(*) from tempty tt where t.i=tt.i) +PREHOOK: type: QUERY +PREHOOK: Input: default@t +PREHOOK: Input: default@tempty +#### A masked pattern was here #### +POSTHOOK: query: select * from t where 0 = (select count(*) from tempty tt where t.i=tt.i) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t +POSTHOOK: Input: default@tempty +#### A masked pattern was here #### +1 1 +3 1 +PREHOOK: query: explain select * from t where 0 = (select min(tt.j) from tempty tt where t.i=tt.i) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from t where 0 = (select min(tt.j) from tempty tt where t.i=tt.i) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int), j (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: tt + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: i is not null (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: min(j) + keys: i (type: int) + mode: hash + outputColumnNames: _col0, _col1 + 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) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 6 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 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: (0 = _col1) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from t where 0 = (select min(tt.j) from tempty tt where t.i=tt.i) +PREHOOK: type: QUERY +PREHOOK: Input: default@t +PREHOOK: Input: default@tempty +#### A masked pattern was here #### +POSTHOOK: query: select * from t where 0 = (select min(tt.j) from tempty tt where t.i=tt.i) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t +POSTHOOK: Input: default@tempty +#### A masked pattern was here #### +PREHOOK: query: drop table t +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t +PREHOOK: Output: default@t +POSTHOOK: query: drop table t +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t +POSTHOOK: Output: default@t +PREHOOK: query: drop table tempty +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tempty +PREHOOK: Output: default@tempty +POSTHOOK: query: drop table tempty +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tempty +POSTHOOK: Output: default@tempty +Warning: Shuffle Join JOIN[15][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select key, count(*) from src group by key having count(*) > + (select count(*) from src s1 group by 4) +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, count(*) from src group by key having count(*) > + (select count(*) from src s1 group by 4) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + 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) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: key (type: string) + mode: hash + 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: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: 4 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 12 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: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 4906 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 > _col2) (type: boolean) + Statistics: Num rows: 83 Data size: 1628 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 1628 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 83 Data size: 1628 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (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) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select key, count(*) from src group by key having count(*) > + (select count(*) from src s1 where s1.key = '90' group by s1.key ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, count(*) from src group by key having count(*) > + (select count(*) from src s1 where s1.key = '90' group by s1.key ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + 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) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: key (type: string) + mode: hash + 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: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = '90') (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: '90' (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 31250 Data size: 695250 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 > _col2) (type: boolean) + Statistics: Num rows: 10416 Data size: 231735 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10416 Data size: 231735 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 10416 Data size: 231735 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + diff --git a/ql/src/test/results/clientpositive/spark/subquery_select.q.out b/ql/src/test/results/clientpositive/spark/subquery_select.q.out new file mode 100644 index 0000000..c3f3d58 --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/subquery_select.q.out @@ -0,0 +1,5379 @@ +Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain SELECT p_size, p_size IN ( + SELECT MAX(p_size) FROM part) +FROM part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, p_size IN ( + SELECT MAX(p_size) FROM part) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 4 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), CASE WHEN ((_col1 = 0)) 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) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3947 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 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + mode: complete + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: SELECT p_size, p_size IN ( + SELECT MAX(p_size) FROM part) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, p_size IN ( + SELECT MAX(p_size) FROM part) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +10 false +12 false +14 false +14 false +23 false +28 false +31 false +42 false +45 false +1 false +2 false +2 false +2 false +2 false +6 false +6 false +7 false +17 false +18 false +19 false +25 false +27 false +34 false +39 false +40 false +46 true +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 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT p_size, p_size IN ( + SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 4 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + 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) + outputColumnNames: _col1, _col3, _col4, _col7 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), CASE WHEN ((_col3 = 0)) THEN (false) WHEN (_col3 is null) THEN (false) WHEN (_col7 is not null) THEN (true) WHEN (_col1 is null) THEN (null) WHEN ((_col4 < _col3)) THEN (null) ELSE (false) END (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 30 Data size: 3807 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 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 726 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: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 7 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT p_size, p_size IN ( + SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, p_size IN ( + SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +7 true +1 true +40 true +23 true +14 true +19 true +2 true +17 false +6 true +10 true +12 true +6 true +46 true +34 true +42 true +31 true +14 true +2 true +2 true +39 true +2 true +27 true +25 true +28 true +18 true +45 true +Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain SELECT p_size, p_size NOT IN ( + SELECT MAX(p_size) FROM part) +FROM part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, p_size NOT IN ( + SELECT MAX(p_size) FROM part) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 4 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), CASE WHEN ((_col1 = 0)) 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) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3947 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 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + mode: complete + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: SELECT p_size, p_size NOT IN ( + SELECT MAX(p_size) FROM part) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, p_size NOT IN ( + SELECT MAX(p_size) FROM part) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +10 true +12 true +14 true +14 true +23 true +28 true +31 true +42 true +45 true +1 true +2 true +2 true +2 true +2 true +6 true +6 true +7 true +17 true +18 true +19 true +25 true +27 true +34 true +39 true +40 true +46 false +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 +PREHOOK: type: QUERY +POSTHOOK: 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 7 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 8 <- Map 7 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 5 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Map 7 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 10 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col3, true) > 0) (type: boolean) + Statistics: Num rows: 9 Data size: 1112 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 1112 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: 9 Data size: 1112 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 9 Data size: 1223 Basic stats: COMPLETE Column stats: NONE + 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: 9 Data size: 1223 Basic stats: COMPLETE Column stats: NONE + value expressions: _col5 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + 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) + outputColumnNames: _col1, _col5, _col6, _col9 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), CASE WHEN ((_col5 = 0)) THEN (true) WHEN (_col5 is null) THEN (true) WHEN (_col9 is not null) THEN (false) WHEN (_col1 is null) THEN (null) WHEN ((_col6 < _col5)) THEN (null) ELSE (true) END (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 14 Data size: 1730 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 + 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: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 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: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 8 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 726 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: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT p_size, p_size NOT IN ( + SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, p_size NOT IN ( + SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +7 false +1 false +40 false +23 false +14 false +19 false +2 false +17 true +6 false +10 false +12 false +6 false +46 false +34 false +42 false +31 false +14 false +2 false +2 false +39 false +2 false +27 false +25 false +28 false +18 false +45 false +Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain SELECT p_size, EXISTS(SELECT p_size FROM part) +FROM part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, EXISTS(SELECT p_size FROM part) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: true (type: boolean) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE 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: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 is not null (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3277 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: boolean) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: SELECT p_size, EXISTS(SELECT p_size FROM part) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, EXISTS(SELECT p_size FROM part) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +2 true +2 true +34 true +6 true +28 true +42 true +14 true +40 true +2 true +25 true +18 true +17 true +14 true +19 true +1 true +45 true +10 true +39 true +27 true +7 true +12 true +31 true +6 true +2 true +46 true +23 true +PREHOOK: query: explain SELECT p_size, EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 3 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col3 is not null (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + 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: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT p_size, EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +1 true +23 true +42 true +14 true +2 true +6 true +2 true +2 true +10 true +2 true +12 true +6 true +18 true +7 true +46 true +40 true +34 true +19 true +17 true +31 true +14 true +39 true +27 true +25 true +28 true +45 true +Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain SELECT p_size, NOT EXISTS(SELECT p_size FROM part) +FROM part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, NOT EXISTS(SELECT p_size FROM part) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: true (type: boolean) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE 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: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 is null (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3277 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: boolean) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: SELECT p_size, NOT EXISTS(SELECT p_size FROM part) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, NOT EXISTS(SELECT p_size FROM part) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +2 false +2 false +34 false +6 false +28 false +42 false +14 false +40 false +2 false +25 false +18 false +17 false +14 false +19 false +1 false +45 false +10 false +39 false +27 false +7 false +12 false +31 false +6 false +2 false +46 false +23 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 +POSTHOOK: query: explain SELECT p_size, NOT EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 3 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col3 is null (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + 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: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT p_size, NOT EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, NOT EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +1 false +23 false +42 false +14 false +2 false +6 false +2 false +2 false +10 false +2 false +12 false +6 false +18 false +7 false +46 false +40 false +34 false +19 false +17 false +31 false +14 false +39 false +27 false +25 false +28 false +45 false +PREHOOK: query: explain select p_size, (select count(p_name) from part p where p.p_type = part.p_name) from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_size, (select count(p_name) from part p where p.p_type = part.p_name) from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(p_name) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), CASE WHEN (_col3 is null) THEN (0) ELSE (_col2) END (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select p_size, (select count(p_name) from part p where p.p_type = part.p_name) from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_size, (select count(p_name) from part p where p.p_type = part.p_name) from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +31 0 +17 0 +34 0 +6 0 +14 0 +39 0 +28 0 +46 0 +27 0 +42 0 +25 0 +18 0 +7 0 +12 0 +23 0 +2 0 +2 0 +14 0 +10 0 +19 0 +1 0 +45 0 +6 0 +2 0 +40 0 +2 0 +PREHOOK: query: explain select p_size, (select max(p_name) from part p where p.p_type = part.p_name) from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_size, (select max(p_name) from part p where p.p_type = part.p_name) from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_name) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), CASE WHEN (_col3 is null) THEN (null) ELSE (_col2) END (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select p_size, (select max(p_name) from part p where p.p_type = part.p_name) from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_size, (select max(p_name) from part p where p.p_type = part.p_name) from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +31 NULL +17 NULL +34 NULL +6 NULL +14 NULL +39 NULL +28 NULL +46 NULL +27 NULL +42 NULL +25 NULL +18 NULL +7 NULL +12 NULL +23 NULL +2 NULL +2 NULL +14 NULL +10 NULL +19 NULL +1 NULL +45 NULL +6 NULL +2 NULL +40 NULL +2 NULL +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain SELECT p_size, (SELECT max(p_size) FROM part) + FROM part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, (SELECT max(p_size) FROM part) + FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3277 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 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: SELECT p_size, (SELECT max(p_size) FROM part) + FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, (SELECT max(p_size) FROM part) + FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +2 46 +2 46 +34 46 +6 46 +28 46 +42 46 +14 46 +40 46 +2 46 +25 46 +18 46 +17 46 +14 46 +19 46 +1 46 +45 46 +10 46 +39 46 +27 46 +7 46 +12 46 +31 46 +6 46 +2 46 +46 46 +23 46 +Warning: Shuffle Join JOIN[13][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain +select * +from src b +where b.key in + (select (select max(key) from src) + from src a + where b.value = a.value and a.key > '9' + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * +from src b +where b.key in + (select (select max(key) from src) + from src a + where b.value = a.value and a.key > '9' + ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 4 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + 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), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key > '9') and value is not null) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Map 6 + 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) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(key) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 550 Data size: 5843 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 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 166 Data size: 32473 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col2 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 32473 Basic stats: COMPLETE Column stats: NONE + 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: 166 Data size: 32473 Basic stats: COMPLETE Column stats: NONE + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 16236 Basic stats: COMPLETE Column stats: NONE + 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: 16236 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[13][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select * +from src b +where b.key in + (select (select max(key) from src) + from src a + where b.value = a.value and a.key > '9' + ) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from src b +where b.key in + (select (select max(key) from src) + from src a + where b.value = a.value and a.key > '9' + ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +98 val_98 +98 val_98 +PREHOOK: query: explain +select * +from src b +where b.key in + (select (select max(key) from src sc where sc.value = a.value) + from src a + where b.value = a.value and a.key > '9' + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * +from src b +where b.key in + (select (select max(key) from src sc where sc.value = a.value) + from src a + where b.value = a.value and a.key > '9' + ) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + 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), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key > '9') and value is not null) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 166 Data size: 1763 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: sc + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: value is not null (type: boolean) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(key) + keys: value (type: string) + mode: hash + 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) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 550 Data size: 5843 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 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: CASE WHEN (_col3 is null) THEN (null) ELSE (_col2) END (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + 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: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 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: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * +from src b +where b.key in + (select (select max(key) from src sc where sc.value = a.value) + from src a + where b.value = a.value and a.key > '9' ) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from src b +where b.key in + (select (select max(key) from src sc where sc.value = a.value) + from src a + where b.value = a.value and a.key > '9' ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +92 val_92 +95 val_95 +95 val_95 +96 val_96 +90 val_90 +90 val_90 +90 val_90 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +PREHOOK: query: CREATE table tnull(i int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tnull +POSTHOOK: query: CREATE table tnull(i int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tnull +PREHOOK: query: insert into tnull values(null) +PREHOOK: type: QUERY +PREHOOK: Output: default@tnull +POSTHOOK: query: insert into tnull values(null) +POSTHOOK: type: QUERY +POSTHOOK: Output: default@tnull +POSTHOOK: Lineage: tnull.i EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select p_size, p_size IN (select i from tnull) from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_size, p_size IN (select i from tnull) from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: tnull + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(i) + 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) + Map 6 + Map Operator Tree: + TableScan + alias: tnull + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: i (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), CASE WHEN ((_col1 = 0)) 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) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3947 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 + 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 + 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) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select p_size, p_size IN (select i from tnull) from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@tnull +#### A masked pattern was here #### +POSTHOOK: query: select p_size, p_size IN (select i from tnull) from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@tnull +#### A masked pattern was here #### +10 NULL +12 NULL +14 NULL +14 NULL +23 NULL +28 NULL +31 NULL +42 NULL +45 NULL +1 NULL +2 NULL +2 NULL +2 NULL +2 NULL +6 NULL +6 NULL +7 NULL +17 NULL +18 NULL +19 NULL +25 NULL +27 NULL +34 NULL +39 NULL +40 NULL +46 NULL +PREHOOK: query: CREATE TABLE tempty(i int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tempty +POSTHOOK: query: CREATE TABLE tempty(i int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tempty +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select p_size, (select count(*) from tempty) from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_size, (select count(*) from tempty) from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: tempty + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3381 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3381 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 + 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 + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select p_size, (select count(*) from tempty) from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@tempty +#### A masked pattern was here #### +POSTHOOK: query: select p_size, (select count(*) from tempty) from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@tempty +#### A masked pattern was here #### +2 0 +2 0 +34 0 +6 0 +28 0 +42 0 +14 0 +40 0 +2 0 +25 0 +18 0 +17 0 +14 0 +19 0 +1 0 +45 0 +10 0 +39 0 +27 0 +7 0 +12 0 +31 0 +6 0 +2 0 +46 0 +23 0 +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain select p_size, (select max(i) from tempty) from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_size, (select max(i) from tempty) from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: tempty + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: i (type: int) + outputColumnNames: i + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: max(i) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3277 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3277 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 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select p_size, (select max(i) from tempty) from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Input: default@tempty +#### A masked pattern was here #### +POSTHOOK: query: select p_size, (select max(i) from tempty) from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Input: default@tempty +#### A masked pattern was here #### +2 NULL +2 NULL +34 NULL +6 NULL +28 NULL +42 NULL +14 NULL +40 NULL +2 NULL +25 NULL +18 NULL +17 NULL +14 NULL +19 NULL +1 NULL +45 NULL +10 NULL +39 NULL +27 NULL +7 NULL +12 NULL +31 NULL +6 NULL +2 NULL +46 NULL +23 NULL +PREHOOK: query: DROP table tempty +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tempty +PREHOOK: Output: default@tempty +POSTHOOK: query: DROP table tempty +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tempty +POSTHOOK: Output: default@tempty +PREHOOK: query: DROP table tnull +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tnull +PREHOOK: Output: default@tnull +POSTHOOK: query: DROP table tnull +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tnull +POSTHOOK: Output: default@tnull +PREHOOK: query: explain SELECT p_size, 1+(SELECT max(p_size) FROM part p + WHERE p.p_type = part.p_type) from part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, 1+(SELECT max(p_size) FROM part p + WHERE p.p_type = part.p_type) from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), (1 + CASE WHEN (_col3 is null) THEN (null) ELSE (_col2) END) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT p_size, 1+(SELECT max(p_size) FROM part p + WHERE p.p_type = part.p_type) from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, 1+(SELECT max(p_size) FROM part p + WHERE p.p_type = part.p_type) from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +1 2 +23 24 +42 43 +14 15 +2 3 +6 7 +2 3 +2 3 +10 11 +2 3 +12 13 +6 7 +18 19 +7 8 +46 47 +40 41 +34 35 +19 20 +17 32 +31 32 +14 15 +39 40 +27 28 +25 26 +28 29 +45 46 +PREHOOK: query: explain SELECT p_size, (SELECT count(p_size) FROM part p + WHERE p.p_type = part.p_type) IS NULL from part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, (SELECT count(p_size) FROM part p + WHERE p.p_type = part.p_type) IS NULL from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), CASE WHEN (_col3 is null) THEN (false) ELSE (_col2 is null) END (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT p_size, (SELECT count(p_size) FROM part p + WHERE p.p_type = part.p_type) IS NULL from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, (SELECT count(p_size) FROM part p + WHERE p.p_type = part.p_type) IS NULL from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +1 false +23 false +42 false +14 false +2 false +6 false +2 false +2 false +10 false +2 false +12 false +6 false +18 false +7 false +46 false +40 false +34 false +19 false +17 false +31 false +14 false +39 false +27 false +25 false +28 false +45 false +Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select p_type, (select p_size from part order by p_size limit 1) = 1 from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_type, (select p_size from part order by p_size limit 1) = 1 from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (SORT, 1) + Reducer 6 <- Reducer 5 (GROUP, 1) + Reducer 8 <- Map 7 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3381 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3381 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col2 + Statistics: Num rows: 26 Data size: 6553 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), (_col2 = 1) (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 6553 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 6553 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 + Reduce Operator Tree: + Select Operator + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 121 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col0) <= 1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reducer 8 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 121 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 121 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select p_type, (select p_size from part order by p_size limit 1) = 1 from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_type, (select p_size from part order by p_size limit 1) = 1 from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +PROMO PLATED TIN true +PROMO PLATED TIN true +LARGE BRUSHED STEEL true +PROMO BURNISHED NICKEL true +STANDARD ANODIZED STEEL true +LARGE BURNISHED STEEL true +MEDIUM ANODIZED COPPER true +ECONOMY POLISHED STEEL true +MEDIUM BURNISHED COPPER true +SMALL POLISHED NICKEL true +STANDARD PLATED TIN true +MEDIUM BURNISHED TIN true +PROMO ANODIZED TIN true +MEDIUM BURNISHED BRASS true +ECONOMY PLATED COPPER true +STANDARD POLISHED STEEL true +SMALL BRUSHED BRASS true +PROMO POLISHED STEEL true +SMALL PLATED STEEL true +ECONOMY BRUSHED COPPER true +STANDARD ANODIZED TIN true +MEDIUM BURNISHED TIN true +STANDARD BURNISHED TIN true +SMALL PLATED BRASS true +ECONOMY BURNISHED STEEL true +LARGE BRUSHED BRASS true +Warning: Shuffle Join JOIN[48][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work 'Reducer 4' 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) +FROM part +PREHOOK: type: QUERY +POSTHOOK: 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) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 13 <- Map 10 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 11 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 13 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 9 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string), p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col2 (type: int) + Map 10 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_name (type: string) + outputColumnNames: p_name + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_name) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + mode: complete + outputColumnNames: _col0, _col1 + 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), _col1 (type: bigint) + Reducer 13 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + 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 + 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) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4, _col5 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + 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: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col4 (type: bigint), _col5 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: string), _col2 (type: int) + 1 _col1 (type: string), _col0 (type: int) + outputColumnNames: _col0, _col2, _col4, _col5, _col8 + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 30 Data size: 3807 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col2 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col8 (type: boolean) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col2, _col4, _col5, _col8, _col9, _col10 + Statistics: Num rows: 30 Data size: 9597 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: 30 Data size: 9597 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col8 (type: boolean), _col9 (type: bigint), _col10 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col2, _col4, _col5, _col8, _col9, _col10, _col12 + Statistics: Num rows: 33 Data size: 10556 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: int), (CASE WHEN ((_col4 = 0)) 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 = 0)) 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) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 33 Data size: 10556 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 33 Data size: 10556 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 7 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 726 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: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 9 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string), _col0 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: string), _col0 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[48][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work 'Reducer 4' 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) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: 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) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +31 true +17 false +34 false +6 false +14 false +39 false +28 false +46 false +27 false +42 false +25 false +18 false +7 false +12 false +23 false +2 false +2 false +14 false +10 false +19 false +1 false +45 false +6 false +2 false +40 false +2 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 +POSTHOOK: query: explain SELECT p_size, NOT EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 3 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col3 is null (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + 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: NONE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT p_size, NOT EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, NOT EXISTS(SELECT p_size FROM part pp where pp.p_type = part.p_type) +FROM part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +1 false +23 false +42 false +14 false +2 false +6 false +2 false +2 false +10 false +2 false +12 false +6 false +18 false +7 false +46 false +40 false +34 false +19 false +17 false +31 false +14 false +39 false +27 false +25 false +28 false +45 false +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 5' is a cross product +Warning: Shuffle Join JOIN[41][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[33][tables = [$hdt$_2, $hdt$_3]] in Work 'Reducer 11' 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 +PREHOOK: type: QUERY +POSTHOOK: 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (PARTITION-LEVEL SORT, 1), Reducer 14 (PARTITION-LEVEL SORT, 1) + Reducer 12 <- Reducer 11 (SORT, 1) + Reducer 14 <- Map 13 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 12 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 10 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Reducer 5 (SORT, 1) + Reducer 7 <- Reducer 6 (GROUP, 1) + Reducer 9 <- Map 13 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 10 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Map 13 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 11 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1 + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Reducer 12 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 9 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: boolean) + Reducer 14 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3589 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), CASE WHEN ((_col1 = 0)) 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) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3947 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3947 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 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1 + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(_col0) + 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) + Reducer 7 + 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 + 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) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 5' is a cross product +Warning: Shuffle Join JOIN[41][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[33][tables = [$hdt$_2, $hdt$_3]] in Work 'Reducer 11' 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 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: 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 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +10 false +12 false +14 false +14 false +23 false +28 false +31 false +42 false +45 false +1 false +2 false +2 false +2 false +2 false +6 false +6 false +7 false +17 false +18 false +19 false +25 false +27 false +34 false +39 false +40 false +46 true +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[28][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain select case when (select count(*) + from part + where p_size between 1 and 20) > 409437 + then (select avg(p_partkey) + from part + where p_partkey between 1 and 20) + else (select max(p_size) + from part + where p_partkey between 10000 and 20000) end sq +from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select case when (select count(*) + from part + where p_size between 1 and 20) > 409437 + then (select avg(p_partkey) + from part + where p_partkey between 1 and 20) + else (select max(p_size) + from part + where p_partkey between 10000 and 20000) end sq +from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 9 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 8 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (GROUP, 1) + Reducer 8 <- Map 7 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_size BETWEEN 1 AND 20 (type: boolean) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Map 7 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey BETWEEN 1 AND 20 (type: boolean) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(p_partkey) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: struct) + Map 9 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey BETWEEN 10000 AND 20000 (type: boolean) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 10 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1 + Statistics: Num rows: 26 Data size: 338 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 338 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 26 Data size: 2340 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 2340 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: double) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 26 Data size: 2470 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: CASE WHEN ((_col1 > 409437)) THEN (_col2) ELSE (_col3) END (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 2470 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 2470 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[25][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[28][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select case when (select count(*) + from part + where p_size between 1 and 20) > 409437 + then (select avg(p_partkey) + from part + where p_partkey between 1 and 20) + else (select max(p_size) + from part + where p_partkey between 10000 and 20000) end sq +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select case when (select count(*) + from part + where p_size between 1 and 20) > 409437 + then (select avg(p_partkey) + from part + where p_partkey between 1 and 20) + else (select max(p_size) + from part + where p_partkey between 10000 and 20000) end sq +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +46.0 +Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: explain select max(p_size) > ( select count(*)-1 from part) from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select max(p_size) > ( select count(*)-1 from part) from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 26 Data size: 3147 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 2 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: (UDFToLong(_col0) > _col1) (type: boolean) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 13 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 + 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 + Select Operator + expressions: (_col0 - 1) (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) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 3' is a cross product +PREHOOK: query: select max(p_size) > ( select count(*)-1 from part) from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select max(p_size) > ( select count(*)-1 from part) from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +true +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: explain select o.p_size, (select count(distinct p_type) from part p where p.p_partkey = o.p_partkey) tmp + FROM part o right join (select * from part where p_size > (select avg(p_size) from part)) t on t.p_partkey = o.p_partkey +PREHOOK: type: QUERY +POSTHOOK: query: explain select o.p_size, (select count(distinct p_type) from part p where p.p_partkey = o.p_partkey) tmp + FROM part o right join (select * from part where p_size > (select avg(p_size) from part)) t on t.p_partkey = o.p_partkey +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 6 <- Map 5 (GROUP, 1) + Reducer 8 <- Map 7 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: o + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: struct) + Map 7 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_partkey (type: int), p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join 0 to 1 + Left Outer Join 0 to 2 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + 2 _col2 (type: int) + outputColumnNames: _col1, _col3, _col4 + Statistics: Num rows: 57 Data size: 6923 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), CASE WHEN (_col4 is null) THEN (0) ELSE (_col3) END (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 57 Data size: 6923 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 57 Data size: 6923 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 5149 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(_col1) > _col2) (type: boolean) + Statistics: Num rows: 8 Data size: 1584 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 8 Data size: 1584 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 8 Data size: 1584 Basic stats: COMPLETE Column stats: NONE + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col1) + keys: _col0 (type: int) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: int) + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 4' is a cross product +PREHOOK: query: select o.p_size, (select count(distinct p_type) from part p where p.p_partkey = o.p_partkey) tmp + FROM part o right join (select * from part where p_size > (select avg(p_size) from part)) t on t.p_partkey = o.p_partkey +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select o.p_size, (select count(distinct p_type) from part p where p.p_partkey = o.p_partkey) tmp + FROM part o right join (select * from part where p_size > (select avg(p_size) from part)) t on t.p_partkey = o.p_partkey +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +39 1 +23 1 +28 1 +25 1 +31 1 +46 1 +27 1 +42 1 +34 1 +45 1 +40 1 +Warning: Shuffle Join JOIN[28][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +Warning: Shuffle Join JOIN[34][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work 'Reducer 4' is a cross product +Warning: Shuffle Join JOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Work 'Reducer 5' is a cross product +PREHOOK: query: explain select (select max(p_size) from part), (select min(p_size) from part), + (select avg(p_size) from part), (select sum(p_size) from part) + from part +PREHOOK: type: QUERY +POSTHOOK: query: explain select (select max(p_size) from part), (select min(p_size) from part), + (select avg(p_size) from part), (select sum(p_size) from part) + from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 13 <- Map 12 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 7 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 11 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Reducer 13 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Map 10 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: avg(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: struct) + Map 12 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 8 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: p_size + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: double) + Reducer 13 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1 + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 234 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 26 Data size: 364 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 364 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int), _col2 (type: int) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 26 Data size: 2366 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 2366 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: double) + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col4 + Statistics: Num rows: 26 Data size: 2600 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col2 (type: int), _col3 (type: double), _col4 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 26 Data size: 2600 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 2600 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 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[28][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product +Warning: Shuffle Join JOIN[34][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work 'Reducer 4' is a cross product +Warning: Shuffle Join JOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Work 'Reducer 5' is a cross product +PREHOOK: query: select (select max(p_size) from part), (select min(p_size) from part), + (select avg(p_size) from part), (select sum(p_size) from part) + from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select (select max(p_size) from part), (select min(p_size) from part), + (select avg(p_size) from part), (select sum(p_size) from part) + from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +46 1 19.692307692307693 512 +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Work '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) + from part t1 +PREHOOK: type: QUERY +POSTHOOK: 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) + from part t1 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3381 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3381 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: int) + 1 _col0 (type: string), _col1 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Work '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) + from part t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: 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) + from part t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +2 28 +2 28 +34 28 +6 28 +28 28 +42 28 +14 28 +40 28 +2 28 +25 28 +18 28 +17 28 +14 28 +19 28 +1 28 +45 28 +10 28 +39 28 +27 28 +7 28 +12 28 +31 28 +6 28 +2 28 +46 28 +23 28 +Warning: Shuffle Join JOIN[44][tables = [$hdt$_0, $hdt$_1]] in Work '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 + group by a1.p_partkey) > 0) + from part t1 +PREHOOK: type: QUERY +POSTHOOK: 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 + group by a1.p_partkey) > 0) + from part t1 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_size (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 10 + Map Operator Tree: + TableScan + alias: a1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(p_size) + keys: p_partkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Map 7 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Map 8 + Map Operator Tree: + TableScan + alias: a1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col2 > 0) (type: boolean) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3381 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3381 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string), _col2 (type: int) + 1 _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + 2 _col0 (type: int) + Statistics: Num rows: 61 Data size: 7614 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: int) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col1) <= 1) (type: boolean) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[44][tables = [$hdt$_0, $hdt$_1]] in Work '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 + group by a1.p_partkey) > 0) + from part t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: 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 + group by a1.p_partkey) > 0) + from part t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +2 28 +2 28 +34 28 +6 28 +28 28 +42 28 +14 28 +40 28 +2 28 +25 28 +18 28 +17 28 +14 28 +19 28 +1 28 +45 28 +10 28 +39 28 +27 28 +7 28 +12 28 +31 28 +6 28 +2 28 +46 28 +23 28 +Warning: Shuffle Join JOIN[69][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work '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 + and (select sum(p_size) from part a1 where a1.p_partkey = p.p_partkey + group by a1.p_partkey) > 0) + from part t1 +PREHOOK: type: QUERY +POSTHOOK: 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 + and (select sum(p_size) from part a1 where a1.p_partkey = p.p_partkey + group by a1.p_partkey) > 0) + from part t1 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Reducer 14 (PARTITION-LEVEL SORT, 2), Reducer 16 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 11 <- Reducer 10 (GROUP, 1) + Reducer 14 <- Map 13 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 16 <- Map 15 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 11 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (GROUP, 2) + Reducer 9 <- Map 12 (PARTITION-LEVEL SORT, 2), Map 8 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int) + Map 12 + Map Operator Tree: + TableScan + alias: pp + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 3147 Basic stats: COMPLETE Column stats: NONE + Map 13 + Map Operator Tree: + TableScan + alias: a1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 15 + Map Operator Tree: + TableScan + alias: a1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(p_size) + keys: p_partkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: t2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: p_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: t2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: p_partkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 8 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (p_size is not null and p_type is not null) (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_partkey (type: int), p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + 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: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 10 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + 2 _col0 (type: int) + Statistics: Num rows: 61 Data size: 7614 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 14 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: int) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col1) <= 1) (type: boolean) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Reducer 16 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col2 > 0) (type: boolean) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 4 Data size: 484 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + Left Outer Join 0 to 2 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + 2 _col1 (type: int) + outputColumnNames: _col1, _col4 + Statistics: Num rows: 57 Data size: 6923 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 57 Data size: 6923 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: int), _col4 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col4, _col6 + Statistics: Num rows: 57 Data size: 7436 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col4 (type: bigint), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 57 Data size: 7436 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 57 Data size: 7436 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 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col0 (type: int) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (sq_count_check(_col1) <= 1) (type: boolean) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 242 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string), _col2 (type: int) + 1 _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[69][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Work '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 + and (select sum(p_size) from part a1 where a1.p_partkey = p.p_partkey + group by a1.p_partkey) > 0) + from part t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: 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 + and (select sum(p_size) from part a1 where a1.p_partkey = p.p_partkey + group by a1.p_partkey) > 0) + from part t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +39 1 28 +10 1 28 +23 1 28 +28 1 28 +14 1 28 +6 1 28 +2 2 28 +2 2 28 +25 1 28 +2 1 28 +31 1 28 +46 1 28 +14 1 28 +7 1 28 +12 1 28 +1 1 28 +6 1 28 +27 1 28 +42 1 28 +34 1 28 +17 1 28 +19 1 28 +45 1 28 +2 1 28 +40 1 28 +18 1 28 +PREHOOK: query: explain SELECT p_size, exp((SELECT max(p_size) FROM part p WHERE p.p_type = part.p_type)) from part +PREHOOK: type: QUERY +POSTHOOK: query: explain SELECT p_size, exp((SELECT max(p_size) FROM part p WHERE p.p_type = part.p_type)) from part +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: p_type (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + 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: int) + Map 3 + Map Operator Tree: + TableScan + alias: p + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_type is not null (type: boolean) + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(p_size) + keys: p_type (type: string) + mode: hash + outputColumnNames: _col0, _col1 + 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: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col2 (type: string) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), exp(CASE WHEN (_col3 is null) THEN (null) ELSE (_col2) END) (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 28 Data size: 3461 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 + 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: 1573 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), true (type: boolean), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1573 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: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT p_size, exp((SELECT max(p_size) FROM part p WHERE p.p_type = part.p_type)) from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: SELECT p_size, exp((SELECT max(p_size) FROM part p WHERE p.p_type = part.p_type)) from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +1 2.718281828459045 +23 9.744803446248903E9 +42 1.73927494152050099E18 +14 1202604.2841647768 +2 7.38905609893065 +6 403.4287934927351 +2 7.38905609893065 +2 7.38905609893065 +10 22026.465794806718 +2 7.38905609893065 +12 162754.79141900392 +6 403.4287934927351 +18 6.565996913733051E7 +7 1096.6331584284585 +46 9.496119420602448E19 +40 2.3538526683702E17 +34 5.834617425274549E14 +19 1.7848230096318725E8 +17 2.9048849665247426E13 +31 2.9048849665247426E13 +14 1202604.2841647768 +39 8.6593400423993744E16 +27 5.3204824060179865E11 +25 7.200489933738588E10 +28 1.446257064291475E12 +45 3.4934271057485095E19 diff --git a/ql/src/test/results/clientpositive/spark/subquery_shared_alias.q.out b/ql/src/test/results/clientpositive/spark/subquery_shared_alias.q.out new file mode 100644 index 0000000..f907f91 --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/subquery_shared_alias.q.out @@ -0,0 +1,23 @@ +PREHOOK: query: select * +from src +where src.key in (select key from src where key > '9') +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from src +where src.key in (select key from src where key > '9') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +92 val_92 +96 val_96 +97 val_97 +97 val_97 +90 val_90 +90 val_90 +90 val_90 +95 val_95 +95 val_95 +98 val_98 +98 val_98 diff --git a/ql/src/test/results/clientpositive/spark/subquery_views.q.out b/ql/src/test/results/clientpositive/spark/subquery_views.q.out new file mode 100644 index 0000000..9a1c25f --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/subquery_views.q.out @@ -0,0 +1,598 @@ +PREHOOK: query: create view cv1 as +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@cv1 +POSTHOOK: query: create view cv1 as +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cv1 +POSTHOOK: Lineage: cv1.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: cv1.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: describe extended cv1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@cv1 +POSTHOOK: query: describe extended cv1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@cv1 +key string +value string + +#### A masked pattern was here #### +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9'), viewExpandedText:select `b`.`key`, `b`.`value` +from `default`.`src` `b` +where exists + (select `a`.`key` + from `default`.`src` `a` + where `b`.`value` = `a`.`value` and `a`.`key` = `b`.`key` and `a`.`value` > 'val_9'), tableType:VIRTUAL_VIEW, rewriteEnabled:false) +PREHOOK: query: select * +from cv1 where cv1.key in (select key from cv1 c where c.key > '95') +PREHOOK: type: QUERY +PREHOOK: Input: default@cv1 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from cv1 where cv1.key in (select key from cv1 c where c.key > '95') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cv1 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +PREHOOK: query: create view cv2 as +select * +from src b +where b.key not in + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_11' + ) +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@cv2 +POSTHOOK: query: create view cv2 as +select * +from src b +where b.key not in + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_11' + ) +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cv2 +POSTHOOK: Lineage: cv2.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: cv2.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: describe extended cv2 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@cv2 +POSTHOOK: query: describe extended cv2 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@cv2 +key string +value string + +#### A masked pattern was here #### +from src b +where b.key not in + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_11' + ), viewExpandedText:select `b`.`key`, `b`.`value` +from `default`.`src` `b` +where `b`.`key` not in + (select `a`.`key` + from `default`.`src` `a` + where `b`.`value` = `a`.`value` and `a`.`key` = `b`.`key` and `a`.`value` > 'val_11' + ), tableType:VIRTUAL_VIEW, rewriteEnabled:false) +PREHOOK: query: explain +select * +from cv2 where cv2.key in (select key from cv2 c where c.key < '11') +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * +from cv2 where cv2.key in (select key from cv2 c where c.key < '11') +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 10 <- Map 18 (GROUP, 2) + Reducer 12 <- Map 11 (PARTITION-LEVEL SORT, 2), Reducer 14 (PARTITION-LEVEL SORT, 2), Reducer 17 (PARTITION-LEVEL SORT, 2) + Reducer 14 <- Map 13 (GROUP, 2) + Reducer 16 <- Map 15 (GROUP, 2) + Reducer 17 <- Reducer 16 (PARTITION-LEVEL SORT, 2), Reducer 19 (PARTITION-LEVEL SORT, 2) + Reducer 19 <- Map 18 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 12 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 13 (GROUP, 2) + Reducer 7 <- Map 15 (GROUP, 2) + Reducer 8 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + properties: + insideView TRUE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < '11') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Map 11 + Map Operator Tree: + TableScan + alias: b + properties: + insideView TRUE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < '11') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Map 13 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((value > 'val_11') and key is not null) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), count(key) + keys: key (type: string), value (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint), _col3 (type: bigint) + Map 15 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((value > 'val_11') and key is not null) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: key (type: string), value (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Map 18 + Map Operator Tree: + TableScan + alias: b + properties: + insideView TRUE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + 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 + Reducer 10 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reducer 12 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + Left Outer Join 0 to 2 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col1 (type: string) + 2 _col4 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col4, _col5, _col9 + Statistics: Num rows: 605 Data size: 6426 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: CASE WHEN ((_col4 = 0)) THEN (true) WHEN (_col4 is null) THEN (true) WHEN (_col9 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (false) ELSE (true) END (type: boolean) + Statistics: Num rows: 302 Data size: 3207 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 302 Data size: 3207 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 302 Data size: 3207 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: 302 Data size: 3207 Basic stats: COMPLETE Column stats: NONE + Reducer 14 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + 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: 881 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint), _col3 (type: bigint) + Reducer 16 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col2, _col3 + Statistics: Num rows: 83 Data size: 881 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: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: string), _col3 (type: boolean) + Reducer 17 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col2, _col3, _col4 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: string), _col2 (type: string) + sort order: ++ + Map-reduce partition columns: _col4 (type: string), _col2 (type: string) + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: boolean) + Reducer 19 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join 0 to 1 + Left Outer Join 0 to 2 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col1 (type: string) + 2 _col4 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1, _col4, _col5, _col9 + Statistics: Num rows: 605 Data size: 6426 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: CASE WHEN ((_col4 = 0)) THEN (true) WHEN (_col4 is null) THEN (true) WHEN (_col9 is not null) THEN (false) WHEN (_col0 is null) THEN (null) WHEN ((_col5 < _col4)) THEN (false) ELSE (true) END (type: boolean) + Statistics: Num rows: 302 Data size: 3207 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 302 Data size: 3207 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: 302 Data size: 3207 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 332 Data size: 3527 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 332 Data size: 3527 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 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + 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: 881 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint), _col3 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col2, _col3 + Statistics: Num rows: 83 Data size: 881 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: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: string), _col3 (type: boolean) + Reducer 8 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col2, _col3, _col4 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col4 (type: string), _col2 (type: string) + sort order: ++ + Map-reduce partition columns: _col4 (type: string), _col2 (type: string) + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + value expressions: _col3 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * +from cv2 where cv2.key in (select key from cv2 c where c.key < '11') +PREHOOK: type: QUERY +PREHOOK: Input: default@cv2 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from cv2 where cv2.key in (select key from cv2 c where c.key < '11') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cv2 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +10 val_10 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +PREHOOK: query: create view cv3 as +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@cv3 +POSTHOOK: query: create view cv3 as +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +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.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 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@cv3 +POSTHOOK: query: describe extended cv3 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@cv3 +key string +value string +_c2 bigint + +#### A masked pattern was here #### +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ), viewExpandedText:select `b`.`key`, `b`.`value`, count(*) +from `default`.`src` `b` +where `b`.`key` in (select `src`.`key` from `default`.`src` where `src`.`key` > '8') +group by `b`.`key`, `b`.`value` +having count(*) in (select count(*) from `default`.`src` `s1` where `s1`.`key` > '9' group by `s1`.`key` ), tableType:VIRTUAL_VIEW, rewriteEnabled:false) +PREHOOK: query: select * from cv3 +PREHOOK: type: QUERY +PREHOOK: Input: default@cv3 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from cv3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cv3 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +80 val_80 1 +82 val_82 1 +83 val_83 2 +84 val_84 2 +85 val_85 1 +86 val_86 1 +87 val_87 1 +9 val_9 1 +90 val_90 3 +92 val_92 1 +95 val_95 2 +96 val_96 1 +97 val_97 2 +98 val_98 2 +PREHOOK: query: select * +from cv3 +where cv3.key in (select key from cv1) +PREHOOK: type: QUERY +PREHOOK: Input: default@cv1 +PREHOOK: Input: default@cv3 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * +from cv3 +where cv3.key in (select key from cv1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cv1 +POSTHOOK: Input: default@cv3 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +90 val_90 3 +92 val_92 1 +95 val_95 2 +96 val_96 1 +97 val_97 2 +98 val_98 2 +PREHOOK: query: drop table tc +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table tc +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table tc (`@d` int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tc +POSTHOOK: query: create table tc (`@d` int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tc +PREHOOK: query: insert overwrite table tc select 1 from src limit 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tc +POSTHOOK: query: insert overwrite table tc select 1 from src limit 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tc +POSTHOOK: Lineage: tc.@d SIMPLE [] +PREHOOK: query: drop view tcv +PREHOOK: type: DROPVIEW +POSTHOOK: query: drop view tcv +POSTHOOK: type: DROPVIEW +PREHOOK: query: create view tcv as select * from tc b where exists (select a.`@d` from tc a where b.`@d`=a.`@d`) +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@tc +PREHOOK: Output: database:default +PREHOOK: Output: default@tcv +POSTHOOK: query: create view tcv as select * from tc b where exists (select a.`@d` from tc a where b.`@d`=a.`@d`) +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@tc +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tcv +POSTHOOK: Lineage: tcv.@d SIMPLE [(tc)b.FieldSchema(name:@d, type:int, comment:null), ] +PREHOOK: query: describe extended tcv +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@tcv +POSTHOOK: query: describe extended tcv +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@tcv +@d int + +#### A masked pattern was here #### +PREHOOK: query: select * from tcv +PREHOOK: type: QUERY +PREHOOK: Input: default@tc +PREHOOK: Input: default@tcv +#### A masked pattern was here #### +POSTHOOK: query: select * from tcv +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tc +POSTHOOK: Input: default@tcv +#### A masked pattern was here #### +1