diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index fb85b9e..35cc08d 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -890,6 +890,7 @@ spark.query.files=add_part_multiple.q, \ escape_distributeby1.q, \ escape_orderby1.q, \ escape_sortby1.q, \ + explainuser_1.q, \ filter_join_breaktask.q, \ filter_join_breaktask2.q, \ groupby1.q, \ diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java index a3a19f4..92225ac 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -210,7 +210,7 @@ public void initialize(HiveConf hiveConf) { if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCORRELATION) && !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEGROUPBYSKEW) && !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME) && - !isTezExecEngine) { + !isTezExecEngine && !isSparkExecEngine) { transformations.add(new CorrelationOptimizer()); } if (HiveConf.getFloatVar(hiveConf, HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE) > 0) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java index f62cf9a..d5fe668 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java @@ -187,8 +187,9 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { && !config.isDependency() && !config.isLogical() && !config.isAuthorize() - && (HiveConf.getBoolVar(ctx.getConf(), HiveConf.ConfVars.HIVE_EXPLAIN_USER) && HiveConf - .getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez"))); + && (HiveConf.getBoolVar(ctx.getConf(), HiveConf.ConfVars.HIVE_EXPLAIN_USER) && (HiveConf + .getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") || HiveConf + .getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")))); ExplainWork work = new ExplainWork(ctx.getResFile(), pCtx, diff --git a/ql/src/test/results/clientpositive/spark/explainuser_1.q.out b/ql/src/test/results/clientpositive/spark/explainuser_1.q.out new file mode 100644 index 0000000..b095967 --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/explainuser_1.q.out @@ -0,0 +1,12741 @@ +PREHOOK: query: explain create table src_orc_merge_test_part(key int, value string) partitioned by (ds string, ts string) stored as orc +PREHOOK: type: CREATETABLE +POSTHOOK: query: explain create table src_orc_merge_test_part(key int, value string) partitioned by (ds string, ts string) stored as orc +POSTHOOK: type: CREATETABLE +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: key int, value string + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + partition columns: ds string, ts string + serde name: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.src_orc_merge_test_part + +PREHOOK: query: create table src_orc_merge_test_part(key int, value string) partitioned by (ds string, ts string) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_orc_merge_test_part +POSTHOOK: query: create table src_orc_merge_test_part(key int, value string) partitioned by (ds string, ts string) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_orc_merge_test_part +PREHOOK: query: alter table src_orc_merge_test_part add partition (ds='2012-01-03', ts='2012-01-03+14:46:31') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Output: default@src_orc_merge_test_part +POSTHOOK: query: alter table src_orc_merge_test_part add partition (ds='2012-01-03', ts='2012-01-03+14:46:31') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Output: default@src_orc_merge_test_part +POSTHOOK: Output: default@src_orc_merge_test_part@ds=2012-01-03/ts=2012-01-03+14%3A46%3A31 +PREHOOK: query: desc extended src_orc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@src_orc_merge_test_part +POSTHOOK: query: desc extended src_orc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@src_orc_merge_test_part +key int +value string +ds string +ts string + +# Partition Information +# col_name data_type comment + +ds string +ts string + +#### A masked pattern was here #### +PREHOOK: query: explain insert overwrite table src_orc_merge_test_part partition(ds='2012-01-03', ts='2012-01-03+14:46:31') select * from src +PREHOOK: type: QUERY +POSTHOOK: query: explain insert overwrite table src_orc_merge_test_part partition(ds='2012-01-03', ts='2012-01-03+14:46:31') select * from src +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Spark +#### 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: UDFToInteger(key) (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.src_orc_merge_test_part + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2012-01-03 + ts 2012-01-03+14:46:31 + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.src_orc_merge_test_part + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: insert overwrite table src_orc_merge_test_part partition(ds='2012-01-03', ts='2012-01-03+14:46:31') select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@src_orc_merge_test_part@ds=2012-01-03/ts=2012-01-03+14%3A46%3A31 +POSTHOOK: query: insert overwrite table src_orc_merge_test_part partition(ds='2012-01-03', ts='2012-01-03+14:46:31') select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@src_orc_merge_test_part@ds=2012-01-03/ts=2012-01-03+14%3A46%3A31 +POSTHOOK: Lineage: src_orc_merge_test_part PARTITION(ds=2012-01-03,ts=2012-01-03+14:46:31).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src_orc_merge_test_part PARTITION(ds=2012-01-03,ts=2012-01-03+14:46:31).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: explain insert into table src_orc_merge_test_part partition(ds='2012-01-03', ts='2012-01-03+14:46:31') select * from src limit 100 +PREHOOK: type: QUERY +POSTHOOK: query: explain insert into table src_orc_merge_test_part partition(ds='2012-01-03', ts='2012-01-03+14:46:31') select * from src limit 100 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (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 + Limit + Number of rows: 100 + Statistics: Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 100 + Statistics: Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.src_orc_merge_test_part + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2012-01-03 + ts 2012-01-03+14:46:31 + replace: false + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.src_orc_merge_test_part + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: explain select count(1) from src_orc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31' +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(1) from src_orc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: 1 + Processor Tree: + ListSink + +PREHOOK: query: explain select sum(hash(key)), sum(hash(value)) from src_orc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31' +PREHOOK: type: QUERY +POSTHOOK: query: explain select sum(hash(key)), sum(hash(value)) from src_orc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31' +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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src_orc_merge_test_part + Statistics: Num rows: 500 Data size: 47000 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(key) (type: int), hash(value) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 47000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_col0), sum(_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 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: alter table src_orc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31') concatenate +PREHOOK: type: ALTER_PARTITION_MERGE +PREHOOK: Input: default@src_orc_merge_test_part +PREHOOK: Output: default@src_orc_merge_test_part@ds=2012-01-03/ts=2012-01-03+14%3A46%3A31 +POSTHOOK: query: alter table src_orc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31') concatenate +POSTHOOK: type: ALTER_PARTITION_MERGE +POSTHOOK: Input: default@src_orc_merge_test_part +POSTHOOK: Output: default@src_orc_merge_test_part@ds=2012-01-03/ts=2012-01-03+14%3A46%3A31 +PREHOOK: query: explain select count(1) from src_orc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31' +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(1) from src_orc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31' +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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src_orc_merge_test_part + Statistics: Num rows: 500 Data size: 47000 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 47000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + 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) + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select sum(hash(key)), sum(hash(value)) from src_orc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31' +PREHOOK: type: QUERY +POSTHOOK: query: explain select sum(hash(key)), sum(hash(value)) from src_orc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31' +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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src_orc_merge_test_part + Statistics: Num rows: 500 Data size: 47000 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(key) (type: int), hash(value) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 47000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_col0), sum(_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 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: drop table src_orc_merge_test_part +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@src_orc_merge_test_part +PREHOOK: Output: default@src_orc_merge_test_part +POSTHOOK: query: drop table src_orc_merge_test_part +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@src_orc_merge_test_part +POSTHOOK: Output: default@src_orc_merge_test_part +Warning: Map Join MAPJOIN[20][bigTable=?] in task 'Stage-1:MAPRED' is a cross product +PREHOOK: query: explain select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +select src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (select * FROM src WHERE src.key < 10) src1 + JOIN + (select * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +select src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (select * FROM src WHERE src.key < 10) src1 + JOIN + (select * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < 10) (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 + Spark HashTable Sink Operator + keys: + 0 + 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (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 + Filter Operator + predicate: (key < 10) (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 + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + input vertices: + 1 Map 4 + Statistics: Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + sort order: ++++ + Statistics: Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(hash(_col0,_col1,_col2,_col3)) + 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: 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, key +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, 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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int), c_float (type: float) + outputColumnNames: key, c_int, c_float + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), ((_col1 + 1) + 2) (type: int), _col3 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 10 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 10 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x +PREHOOK: type: QUERY +POSTHOOK: query: explain select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x +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 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int), c_float (type: float) + outputColumnNames: key, c_int, c_float + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: bigint), (((UDFToFloat(_col1) + _col2) + 1.0) + 2.0) (type: float) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col0 (type: bigint), _col1 (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint), _col1 (type: float) + sort order: ++ + Map-reduce partition columns: _col0 (type: bigint), _col1 (type: float) + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint), KEY._col1 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: float), _col0 (type: bigint), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c +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 (SORT, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 11 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 2) + Reducer 7 <- Reducer 6 (SORT, 1) + Reducer 9 <- Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and key is not null) (type: boolean) + Statistics: Num rows: 5 Data size: 465 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 2 Data size: 202 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 2 Data size: 202 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Map 11 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 8 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and key is not null) (type: boolean) + Statistics: Num rows: 5 Data size: 465 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 2 Data size: 202 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 2 Data size: 202 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 10 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 + _col4) >= 0) (type: boolean) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col2, _col6 + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 > 0) or (_col6 >= 0)) (type: boolean) + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: bigint), _col6 (type: int) + outputColumnNames: _col2, _col6 + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col2 (type: bigint), _col6 (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: bigint), _col1 (type: int) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col0 (type: bigint), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: (UDFToLong(_col0) + _col1) (type: bigint), _col1 (type: bigint) + sort order: -+ + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: bigint), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint), (UDFToDouble((_col1 + 1)) / 10.0) (type: double) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: double), _col2 (type: bigint) + sort order: -+ + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc +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 (SORT, 1) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 10 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 2) + Reducer 7 <- Reducer 6 (SORT, 1) + Reducer 9 <- Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0) and key is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Map 10 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((c_int > 0) and key is not null) (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 8 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0) and key is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint), (UDFToLong((_col1 + 1)) % _col3) (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: bigint), _col1 (type: int) + sort order: +- + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey1 (type: int), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 + _col4) >= 0) (type: boolean) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col2, _col6 + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col1 > 0) or (_col6 >= 0)) and ((_col6 >= 1) or (_col2 >= 1)) and ((UDFToLong(_col6) + _col2) >= 0)) (type: boolean) + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: bigint), _col6 (type: int) + outputColumnNames: _col2, _col6 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col6 (type: int), _col2 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: (UDFToLong(_col0) % _col1) (type: bigint), _col0 (type: int) + sort order: +- + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) cbo_t1 right outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 2) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) cbo_t1 right outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 2) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c +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 (SORT, 1) + Reducer 4 <- Map 8 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + 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: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean) + Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean) + Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Map 8 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint), (UDFToLong((_col1 + 1)) + _col3) (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: bigint), _col0 (type: string) + sort order: +- + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + Right Outer Join0 to 2 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + outputColumnNames: _col1, _col2, _col4, _col6 + Statistics: Num rows: 4 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col1 + _col4) >= 2) and ((_col1 > 0) or (_col6 >= 0))) (type: boolean) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: bigint), _col6 (type: int) + outputColumnNames: _col2, _col6 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col2 (type: bigint), _col6 (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: bigint), _col1 (type: int) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col0 (type: bigint), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) cbo_t1 full outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) cbo_t2 on cbo_t1.a=p full outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) cbo_t1 full outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) cbo_t2 on cbo_t1.a=p full outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int +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 (SORT, 1) + Reducer 4 <- Map 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (GROUP, 2) + Reducer 6 <- Reducer 5 (SORT, 1) + Reducer 8 <- Map 7 (GROUP, 2) + Reducer 9 <- Reducer 8 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean) + Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Map 10 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (c_int > 0) (type: boolean) + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 7 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0)) (type: boolean) + Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint), (UDFToDouble(_col3) + UDFToDouble(_col0)) (type: double) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: double) + sort order: - + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: int), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int), VALUE._col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + Right Outer Join0 to 2 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + outputColumnNames: _col1, _col2, _col4, _col6 + Statistics: Num rows: 4 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col1 + _col4) >= 0) and ((_col1 > 0) or (_col6 >= 0)) and ((_col6 >= 1) or (_col2 >= 1)) and ((UDFToLong(_col6) + _col2) >= 0)) (type: boolean) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: bigint), _col6 (type: int) + outputColumnNames: _col2, _col6 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col6 (type: int), _col2 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint), (UDFToDouble(_col0) + UDFToDouble((_col1 + 1))) (type: double) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: double), _col2 (type: bigint) + sort order: -+ + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: int) + Reducer 9 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c +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, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 8 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + 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: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0) and key is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0) and key is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 186 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Map 8 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 + _col4) >= 0) (type: boolean) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col2, _col6 + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 > 0) or (_col6 >= 0)) (type: boolean) + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: bigint), _col6 (type: int) + outputColumnNames: _col2, _col6 + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col2 (type: bigint), _col6 (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: bigint), _col1 (type: int) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col0 (type: bigint), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select unionsrc.key FROM (select 'tst1' as key, count(1) as value from src) unionsrc +PREHOOK: type: QUERY +POSTHOOK: query: explain select unionsrc.key FROM (select 'tst1' as key, count(1) as value from src) unionsrc +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: 1 + Processor Tree: + ListSink + +PREHOOK: query: explain select unionsrc.key FROM (select 'max' as key, max(c_int) as value from cbo_t3 s1 + UNION ALL + select 'min' as key, min(c_int) as value from cbo_t3 s2 + UNION ALL + select 'avg' as key, avg(c_int) as value from cbo_t3 s3) unionsrc order by unionsrc.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select unionsrc.key FROM (select 'max' as key, max(c_int) as value from cbo_t3 s1 + UNION ALL + select 'min' as key, min(c_int) as value from cbo_t3 s2 + UNION ALL + select 'avg' as key, avg(c_int) as value from cbo_t3 s3) unionsrc order by unionsrc.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, 1) + Reducer 3 <- Reducer 2 (SORT, 1), Reducer 5 (SORT, 1), Reducer 7 (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: s1 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(key) + 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 4 + Map Operator Tree: + TableScan + alias: s2 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(key) + 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: s3 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(key) + 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: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: 'max' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 3 Data size: 261 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 261 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 261 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + 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: 'min' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 3 Data size: 261 Basic stats: COMPLETE Column stats: COMPLETE + 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: COMPLETE + Select Operator + expressions: 'avg' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 3 Data size: 261 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from cbo_t3 s1 + UNION ALL + select 'min' as key, min(c_int) as value from cbo_t3 s2 + UNION ALL + select 'avg' as key, avg(c_int) as value from cbo_t3 s3) unionsrc group by unionsrc.key order by unionsrc.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from cbo_t3 s1 + UNION ALL + select 'min' as key, min(c_int) as value from cbo_t3 s2 + UNION ALL + select 'avg' as key, avg(c_int) as value from cbo_t3 s3) unionsrc group by unionsrc.key order by unionsrc.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, 1) + Reducer 3 <- Reducer 2 (GROUP, 2), Reducer 6 (GROUP, 2), Reducer 8 (GROUP, 2) + Reducer 4 <- Reducer 3 (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: s1 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(key) + 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: s2 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(key) + 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 7 + Map Operator Tree: + TableScan + alias: s3 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(key) + 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: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: 'max' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + 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: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + 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 + Select Operator + expressions: 'min' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Reducer 8 + 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: 'avg' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t1.key from cbo_t1 join cbo_t3 where cbo_t1.key=cbo_t3.key and cbo_t1.key >= 1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t1.key from cbo_t1 join cbo_t3 where cbo_t1.key=cbo_t3.key and cbo_t1.key >= 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), Map 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (UDFToDouble(key) >= 1.0) (type: boolean) + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Map 3 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (UDFToDouble(key) >= 1.0) (type: boolean) + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 18 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 18 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t1.c_int, cbo_t2.c_int from cbo_t1 left outer join cbo_t2 on cbo_t1.key=cbo_t2.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t1.c_int, cbo_t2.c_int from cbo_t1 left outer join cbo_t2 on cbo_t1.key=cbo_t2.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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col3 + Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t1.c_int, cbo_t2.c_int from cbo_t1 full outer join cbo_t2 on cbo_t1.key=cbo_t2.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t1.c_int, cbo_t2.c_int from cbo_t1 full outer join cbo_t2 on cbo_t1.key=cbo_t2.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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col3 + Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select b, cbo_t1.c, cbo_t2.p, q, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1) cbo_t1 join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key +PREHOOK: type: QUERY +POSTHOOK: query: explain select b, cbo_t1.c, cbo_t2.p, q, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1) cbo_t1 join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=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 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1581 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int), c_float (type: float) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 18 Data size: 1581 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1581 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: float) + Map 3 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + 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 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + outputColumnNames: _col1, _col2, _col4, _col5, _col6 + Statistics: Num rows: 291 Data size: 29391 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col2 (type: float), _col5 (type: string), _col6 (type: int), _col4 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 291 Data size: 29391 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 291 Data size: 29391 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select key, cbo_t1.c_int, cbo_t2.p, q from cbo_t1 join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2) cbo_t2 on cbo_t1.key=p join (select key as a, c_int as b, cbo_t3.c_float as c from cbo_t3)cbo_t3 on cbo_t1.key=a +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, cbo_t1.c_int, cbo_t2.p, q from cbo_t1 join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2) cbo_t2 on cbo_t1.key=p join (select key as a, c_int as b, cbo_t3.c_float as c from cbo_t3)cbo_t3 on cbo_t1.key=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), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1445 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 18 Data size: 1445 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1445 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + 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 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 291 Data size: 51798 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), _col3 (type: string), _col4 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 291 Data size: 51798 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 291 Data size: 51798 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from (select q, b, cbo_t2.p, cbo_t1.c, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 full outer join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q == 2) and (b > 0 or c_int >= 0)) R where (q + 1 = 2) and (R.b > 0 or c_int >= 0) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from (select q, b, cbo_t2.p, cbo_t1.c, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 full outer join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q == 2) and (b > 0 or c_int >= 0)) R where (q + 1 = 2) and (R.b > 0 or c_int >= 0) +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 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0)) and key is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int), c_float (type: float) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: float) + Map 4 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0)) and key is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 801 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 9 Data size: 801 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 40 Data size: 7280 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col1 + _col4) = 2) and ((_col4 + 1) = 2)) (type: boolean) + Statistics: Num rows: 10 Data size: 1820 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 10 Data size: 1820 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: float), _col3 (type: string), _col4 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col2, _col3, _col4, _col6 + Statistics: Num rows: 36 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 > 0) or (_col6 >= 0)) (type: boolean) + Statistics: Num rows: 36 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col4 (type: int), _col1 (type: int), _col3 (type: string), _col2 (type: float), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 36 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 36 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from (select q, b, cbo_t2.p, cbo_t1.c, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 right outer join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q == 2) and (b > 0 or c_int >= 0)) R where (q + 1 = 2) and (R.b > 0 or c_int >= 0) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from (select q, b, cbo_t2.p, cbo_t1.c, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 right outer join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q == 2) and (b > 0 or c_int >= 0)) R where (q + 1 = 2) and (R.b > 0 or c_int >= 0) +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 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0))) (type: boolean) + Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int), c_float (type: float) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: float) + Map 3 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0))) (type: boolean) + Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 890 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 10 Data size: 890 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + Right Outer Join0 to 2 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + outputColumnNames: _col1, _col2, _col3, _col4, _col6 + Statistics: Num rows: 200 Data size: 20200 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col1 + _col4) = 2) and ((_col1 > 0) or (_col6 >= 0)) and ((_col4 + 1) = 2)) (type: boolean) + Statistics: Num rows: 50 Data size: 5050 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col4 (type: int), _col1 (type: int), _col3 (type: string), _col2 (type: float), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 50 Data size: 5050 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 50 Data size: 5050 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, key order by x limit 1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, key order by x 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 (GROUP, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int), c_float (type: float) + outputColumnNames: key, c_int, c_float + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), ((_col1 + 1) + 2) (type: int), _col3 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 10 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Statistics: Num rows: 10 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: int), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 10 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 1 + Processor Tree: + ListSink + +PREHOOK: query: explain select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x order by x,y limit 1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x order by x,y 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 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) + Reducer 4 <- Reducer 3 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int), c_float (type: float) + outputColumnNames: key, c_int, c_float + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: bigint), (((UDFToFloat(_col1) + _col2) + 1.0) + 2.0) (type: float) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col1 (type: float), _col0 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: float), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: float), _col1 (type: bigint) + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: float), KEY._col1 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: float), _col1 (type: bigint) + sort order: ++ + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: float), KEY.reducesinkkey1 (type: bigint), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 100 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 1 + Processor Tree: + ListSink + +PREHOOK: query: explain select key from(select key from (select key from cbo_t1 limit 5)cbo_t2 limit 5)cbo_t3 limit 5 +PREHOOK: type: QUERY +POSTHOOK: query: explain select key from(select key from (select key from cbo_t1 limit 5)cbo_t2 limit 5)cbo_t3 limit 5 +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 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 5 + Processor Tree: + ListSink + +PREHOOK: query: explain select key, c_int from(select key, c_int from (select key, c_int from cbo_t1 order by c_int limit 5)cbo_t1 order by c_int limit 5)cbo_t2 order by c_int limit 5 +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, c_int from(select key, c_int from (select key, c_int from cbo_t1 order by c_int limit 5)cbo_t1 order by c_int limit 5)cbo_t2 order by c_int limit 5 +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 (SORT, 1) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 3 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 5 Data size: 445 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Statistics: Num rows: 5 Data size: 445 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 445 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 5 Data size: 445 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Statistics: Num rows: 5 Data size: 445 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 445 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 5 Data size: 445 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 445 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 5 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a limit 5) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc limit 5) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c limit 5 +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a limit 5) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc limit 5) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c limit 5 +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 (SORT, 1) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Reducer 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 11 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Reducer 5 (GROUP, 2) + Reducer 7 <- Reducer 6 (SORT, 1) + Reducer 9 <- Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0))) (type: boolean) + Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 303 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 3 Data size: 303 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Map 11 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 8 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0))) (type: boolean) + Statistics: Num rows: 6 Data size: 558 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 303 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 3 Data size: 303 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 10 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 2 Data size: 210 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 210 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 303 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3 Data size: 291 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 3 Data size: 291 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3 Data size: 291 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 3 Data size: 291 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 2 Data size: 202 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 + _col4) >= 0) (type: boolean) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col2, _col6 + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 > 0) or (_col6 >= 0)) (type: boolean) + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: bigint), _col6 (type: int) + outputColumnNames: _col2, _col6 + Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col2 (type: bigint), _col6 (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: bigint), _col1 (type: int) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col0 (type: bigint), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: (UDFToLong(_col0) + _col1) (type: bigint), _col1 (type: bigint) + sort order: -+ + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: int), _col2 (type: bigint) + Reducer 7 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: bigint), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 5 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 303 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (_col1 + 1) (type: int), _col3 (type: bigint), (UDFToDouble((_col1 + 1)) / 10.0) (type: double) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: double), _col2 (type: bigint) + sort order: -+ + Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: int) + + Stage: Stage-0 + Fetch Operator + limit: 5 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t1.c_int from cbo_t1 left semi join cbo_t2 on cbo_t1.key=cbo_t2.key where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t1.c_int from cbo_t1 left semi join cbo_t2 on cbo_t1.key=cbo_t2.key where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) +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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0)) and key is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1445 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 18 Data size: 1445 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE + 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: _col1 + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from (select c, b, a from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c >= 0)) R where (b + 1 = 2) and (R.b > 0 or c >= 0) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from (select c, b, a from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c >= 0)) R where (b + 1 = 2) and (R.b > 0 or c >= 0) +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 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0)) and key is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), c_int (type: int), c_float (type: float) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: float) + Map 3 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0)) and key is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 837 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 9 Data size: 765 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 255 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 3 Data size: 255 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 18 Data size: 1445 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 18 Data size: 1445 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + Left Semi Join 0 to 2 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 16 Data size: 1488 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: float), _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 16 Data size: 1488 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 16 Data size: 1488 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc) cbo_t1 left semi join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 >= 0) and (b > 0 or a >= 0) group by a, c having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, a +PREHOOK: type: QUERY +POSTHOOK: query: explain select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc) cbo_t1 left semi join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 >= 0) and (b > 0 or a >= 0) group by a, c having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, 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 (GROUP, 2) + Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 4 <- Map 10 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (GROUP, 2) + Reducer 6 <- Reducer 5 (SORT, 1) + Reducer 8 <- Map 7 (GROUP, 2) + Reducer 9 <- Reducer 8 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0) and (((c_int + 1) + 1) >= 0) and (((c_int + 1) > 0) or (UDFToDouble(key) >= 0.0)) and (UDFToDouble(key) > 0.0)) (type: boolean) + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Map 10 + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (UDFToDouble(key) > 0.0) (type: boolean) + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 255 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 3 Data size: 255 Basic stats: COMPLETE Column stats: COMPLETE + Map 7 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 1767 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0.0) and (UDFToDouble(key) > 0.0)) (type: boolean) + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(c_int) + keys: key (type: string), c_int (type: int), c_float (type: float) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: float) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: float) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col0 (type: string), _col3 (type: bigint) + outputColumnNames: _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((UDFToDouble(_col2) >= 1.0) or (_col3 >= 1)) and ((UDFToDouble(_col2) + UDFToDouble(_col3)) >= 0.0)) (type: boolean) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col3 (type: bigint), (UDFToDouble(_col2) + UDFToDouble((_col1 + 1))) (type: double) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: double), _col1 (type: bigint) + sort order: -+ + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + Left Semi Join 0 to 2 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col1 (type: bigint), _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: bigint), _col1 (type: string) + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string), _col0 (type: bigint), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: bigint), _col0 (type: string) + sort order: ++ + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: float) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), (UDFToDouble((_col1 + 1)) + (UDFToDouble(_col3) / 10.0)) (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: double), _col0 (type: string) + sort order: -+ + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 9 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 85 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 85 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 85 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select cbo_t1.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select cbo_t1.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + TableScan + alias: cbo_t1 + Select Operator + expressions: key (type: string), c_int (type: int), (((UDFToFloat(c_int) + c_float) * 10.0) + 5.0) (type: float) + outputColumnNames: _col0, _col1, _col2 + ListSink + +PREHOOK: query: explain select null from cbo_t1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select null from cbo_t1 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + TableScan + alias: cbo_t1 + Select Operator + expressions: null (type: string) + outputColumnNames: _col0 + ListSink + +PREHOOK: query: explain select key from cbo_t1 where c_int = -6 or c_int = +6 +PREHOOK: type: QUERY +POSTHOOK: query: explain select key from cbo_t1 where c_int = -6 or c_int = +6 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + TableScan + alias: cbo_t1 + Filter Operator + predicate: ((c_int = -6) or (c_int = 6)) (type: boolean) + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + ListSink + +PREHOOK: query: explain select count(cbo_t1.dt) from cbo_t1 join cbo_t2 on cbo_t1.dt = cbo_t2.dt where cbo_t1.dt = '2014' +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(cbo_t1.dt) from cbo_t1 join cbo_t2 on cbo_t1.dt = cbo_t2.dt where cbo_t1.dt = '2014' +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 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 422 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 20 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 20 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: cbo_t2 + Statistics: Num rows: 20 Data size: 422 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 20 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 20 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + Statistics: Num rows: 400 Data size: 3200 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count('2014') + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 3 + 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 + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * +from src_cbo b +where not exists + (select distinct a.key + from src_cbo a + where b.value = a.value and a.value > 'val_2' + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * +from src_cbo b +where not exists + (select distinct a.key + from src_cbo a + where b.value = a.value and a.value > 'val_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, 2), Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((value = value) and (value > 'val_2')) (type: boolean) + Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: value (type: string), key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col3 + Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col3 is null (type: boolean) + Statistics: Num rows: 1 Data size: 182 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col1 (type: string) + mode: complete + outputColumnNames: _col0 + Statistics: Num rows: 36 Data size: 3276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 36 Data size: 3420 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 36 Data size: 3420 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * +from src_cbo b +group by key, value +having not exists + (select a.key + from src_cbo a + where b.value = a.value and a.key = b.key and a.value > 'val_12' + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select * +from src_cbo b +group by key, value +having not exists + (select a.key + from src_cbo a + where b.value = a.value and a.key = b.key and a.value > 'val_12' + ) +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, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: key, value + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string), value (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((value = value) and (key = key) and (value > 'val_12')) (type: boolean) + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string), value (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 3560 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 20 Data size: 3560 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + 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: 44500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col4 + Statistics: Num rows: 250 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col4 is null (type: boolean) + Statistics: Num rows: 1 Data size: 182 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 3560 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 20 Data size: 3640 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 20 Data size: 3640 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: create view cv1 as +select * +from src_cbo b +where exists + (select a.key + from src_cbo a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@src_cbo +PREHOOK: Output: database:default +PREHOOK: Output: default@cv1 +POSTHOOK: query: create view cv1 as +select * +from src_cbo b +where exists + (select a.key + from src_cbo a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@src_cbo +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cv1 +POSTHOOK: Lineage: cv1.key SIMPLE [(src_cbo)b.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: cv1.value SIMPLE [(src_cbo)b.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: explain select * from cv1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from cv1 +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 3 (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: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Map 3 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((value = value) and (key = key) and (value > 'val_9')) (type: boolean) + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 3560 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 20 Data size: 3560 Basic stats: COMPLETE Column stats: COMPLETE + 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: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * +from (select * + from src_cbo b + where exists + (select a.key + from src_cbo a + where b.value = a.value and a.key = b.key and a.value > 'val_9') + ) a +PREHOOK: type: QUERY +POSTHOOK: query: explain select * +from (select * + from src_cbo b + where exists + (select a.key + from src_cbo a + where b.value = a.value and a.key = b.key and a.value > 'val_9') + ) 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), Map 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Map 3 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((value = value) and (key = key) and (value > 'val_9')) (type: boolean) + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 41 Data size: 7298 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 3560 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 20 Data size: 3560 Basic stats: COMPLETE Column stats: COMPLETE + 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: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * +from src_cbo +where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') +PREHOOK: type: QUERY +POSTHOOK: query: explain select * +from src_cbo +where src_cbo.key in (select key from src_cbo 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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src_cbo + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + 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: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +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) +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) +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 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: l_partkey is not null (type: boolean) + Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Map 5 + Map Operator Tree: + TableScan + alias: li + Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((l_linenumber = 1) and l_partkey is not null) (type: boolean) + Statistics: Num rows: 17 Data size: 272 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 17 Data size: 272 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 17 Data size: 272 Basic stats: COMPLETE Column stats: COMPLETE + 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: 9600 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((l_shipmode = 'AIR') and (l_linenumber = l_linenumber)) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: l_orderkey (type: int), l_linenumber (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int), _col1 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + 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: 5 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int), _col4 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: int), _col4 (type: int) + Statistics: Num rows: 5 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE + 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: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col3 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select key, value, count(*) +from src_cbo b +where b.key in (select key from src_cbo where src_cbo.key > '8') +group by key, value +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, value, count(*) +from src_cbo b +where b.key in (select key from src_cbo where src_cbo.key > '8') +group by key, value +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' 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), Reducer 6 (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 6 <- Map 5 (GROUP, 2) + Reducer 8 <- Map 7 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '8') (type: boolean) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: src_cbo + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '8') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + Map 7 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '9') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 69 Data size: 6555 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 69 Data size: 6555 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE + 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: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col2 is not null (type: boolean) + Statistics: Num rows: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: bigint) + sort order: + + Map-reduce partition columns: _col2 (type: bigint) + Statistics: Num rows: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE + 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: 34 Data size: 6324 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 34 Data size: 6324 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 6 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + 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: 69 Data size: 6555 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 34 Data size: 272 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 34 Data size: 272 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select p_mfgr, p_name, avg(p_size) +from part +group by p_mfgr, p_name +having p_name in + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) +PREHOOK: type: QUERY +POSTHOOK: query: explain select p_mfgr, p_name, avg(p_size) +from part +group by p_mfgr, p_name +having p_name in + (select first_value(p_name) over(partition by p_mfgr order by 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 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: avg(p_size) + keys: p_name (type: string), p_mfgr (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 3835 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 13 Data size: 3835 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: struct) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_size (type: int) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 2951 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string), _col0 (type: string), _col2 (type: double) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 2951 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 13 Data size: 2951 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col2 (type: double) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 1362 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 6 Data size: 1362 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + 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: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: first_value_window_0 + arguments: _col1 + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: first_value_window_0 is not null (type: boolean) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: first_value_window_0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 26 Data size: 4784 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * +from src_cbo +where src_cbo.key not in + ( select key from src_cbo s1 + where s1.key > '2' + ) order by key +PREHOOK: type: QUERY +POSTHOOK: query: explain select * +from src_cbo +where src_cbo.key not in + ( select key from src_cbo s1 + where s1.key > '2' + ) order by 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, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (SORT, 1) + 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: src_cbo + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '2') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(key) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Map 7 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key > '2') (type: boolean) + Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + 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: 97000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 97000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: bigint), _col3 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Statistics: Num rows: 500 Data size: 99000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col2 = 0) or (_col5 is null and _col0 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 500 Data size: 99000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 6003 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 69 Data size: 6279 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 69 Data size: 6279 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +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 as r from part) a + where r < 10 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 as r from part) a + where r < 10 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 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: b + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col2 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((p_size < 10) and (p_mfgr = p_mfgr)) (type: boolean) + Statistics: Num rows: 4 Data size: 892 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_name (type: string), p_mfgr (type: string) + outputColumnNames: p_name, p_mfgr + Statistics: Num rows: 4 Data size: 892 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(p_name) + keys: p_mfgr (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((p_size < 10) and (p_mfgr = p_mfgr)) (type: boolean) + Statistics: Num rows: 4 Data size: 892 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_name (type: string), p_mfgr (type: string) + outputColumnNames: p_name, p_mfgr + Statistics: Num rows: 4 Data size: 892 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: p_name (type: string), p_mfgr (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 438 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 2 Data size: 438 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: int), _col4 (type: bigint), _col5 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 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: 26 Data size: 6318 Basic stats: COMPLETE Column stats: COMPLETE + 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: 13 Data size: 3159 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string), _col0 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 2899 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 2899 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + 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: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint), _col2 (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: 2 Data size: 438 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col0 = _col0) (type: boolean) + Statistics: Num rows: 1 Data size: 219 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 223 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 1 Data size: 223 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size from part) a + where p_size < 10 + ) order by p_name +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 from part) a + where p_size < 10 + ) order by 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, 1), Reducer 6 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (SORT, 1) + Reducer 6 <- Map 5 (GROUP, 1) + Reducer 8 <- Map 5 (GROUP, 1) + 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: 3250 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_name (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (p_size < 10) (type: boolean) + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: avg(p_size) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Statistics: Num rows: 26 Data size: 3666 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToDouble(_col1) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col1) (type: double) + Statistics: Num rows: 26 Data size: 3666 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 UDFToDouble(_col1) (type: double) + 1 _col0 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col5 + Statistics: Num rows: 26 Data size: 3770 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col2 = 0) or (_col5 is null and _col1 is not null and (_col3 >= _col2))) (type: boolean) + Statistics: Num rows: 26 Data size: 3770 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: double) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: double), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a + where min(p_retailprice) = l and r - l > 600 + ) + order by b.p_mfgr +PREHOOK: type: QUERY +POSTHOOK: query: explain select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a + where min(p_retailprice) = l and r - l > 600 + ) + order by b.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 <- Map 6 (GROUP, 2) + Reducer 11 <- Reducer 10 (GROUP, 2) + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 8 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 11 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (SORT, 1) + Reducer 7 <- Map 6 (GROUP, 2) + Reducer 8 <- Reducer 7 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_mfgr (type: string), p_retailprice (type: double) + outputColumnNames: p_mfgr, p_retailprice + Statistics: Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(p_retailprice) + keys: p_mfgr (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: double) + Map 6 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_mfgr (type: string), p_retailprice (type: double) + outputColumnNames: p_mfgr, p_retailprice + Statistics: Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(p_retailprice), max(p_retailprice) + keys: p_mfgr (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: double), _col2 (type: double) + Reducer 10 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 = _col1) and ((_col2 - _col1) > 600.0)) (type: boolean) + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string), _col1 (type: double) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: double) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: double) + Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 11 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: double) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col0 = _col0) (type: boolean) + Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: double), true (type: boolean) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 110 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: double) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: double) + Statistics: Num rows: 1 Data size: 110 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: double) + sort order: + + Map-reduce partition columns: _col1 (type: double) + Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col1 (type: double) + 1 _col0 (type: double) + outputColumnNames: _col0, _col1, _col3, _col4 + Statistics: Num rows: 5 Data size: 610 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: double) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: double) + Statistics: Num rows: 5 Data size: 610 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint), _col4 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: double) + 1 _col0 (type: string), _col1 (type: double) + outputColumnNames: _col0, _col1, _col3, _col4, _col7 + Statistics: Num rows: 5 Data size: 630 Basic stats: COMPLETE Column stats: COMPLETE + 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: 3 Data size: 378 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 318 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 3 Data size: 318 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: double) + Reducer 5 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 318 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 318 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col1 = _col1) and ((_col2 - _col1) > 600.0)) (type: boolean) + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + keys: _col1 (type: double) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: double) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select count(c_int) over(), sum(c_float) over(), max(c_int) over(), min(c_int) over(), row_number() over(), rank() over(), dense_rank() over(), percent_rank() over(), lead(c_int, 2, c_int) over(), lag(c_float, 2, c_float) over() from cbo_t1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(c_int) over(), sum(c_float) over(), max(c_int) over(), min(c_int) over(), row_number() over(), rank() over(), dense_rank() over(), percent_rank() over(), lead(c_int, 2, c_int) over(), lag(c_float, 2, c_float) over() from cbo_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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: 0 (type: int) + sort order: + + Map-reduce partition columns: 0 (type: int) + Statistics: Num rows: 20 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: c_int (type: int), c_float (type: float) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col2 (type: int), VALUE._col3 (type: float) + outputColumnNames: _col2, _col3 + Statistics: Num rows: 20 Data size: 9192 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: int, _col3: float + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: 0 ASC NULLS FIRST + partition by: 0 + raw input shape: + window functions: + window function definition + alias: count_window_0 + arguments: _col2 + name: count + window function: GenericUDAFCountEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: sum_window_1 + arguments: _col3 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: max_window_2 + arguments: _col2 + name: max + window function: GenericUDAFMaxEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: min_window_3 + arguments: _col2 + name: min + window function: GenericUDAFMinEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: row_number_window_4 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: rank_window_5 + arguments: 0 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_6 + arguments: 0 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: percent_rank_window_7 + arguments: 0 + name: percent_rank + window function: GenericUDAFPercentRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: lead_window_8 + arguments: _col2, 2, _col2 + name: lead + window function: GenericUDAFLeadEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: lag_window_9 + arguments: _col3, 2, _col3 + name: lag + window function: GenericUDAFLagEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 20 Data size: 9192 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: count_window_0 (type: bigint), sum_window_1 (type: double), max_window_2 (type: int), min_window_3 (type: int), row_number_window_4 (type: int), rank_window_5 (type: int), dense_rank_window_6 (type: int), percent_rank_window_7 (type: double), lead_window_8 (type: int), lag_window_9 (type: float) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 20 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 20 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from (select count(c_int) over(), sum(c_float) over(), max(c_int) over(), min(c_int) over(), row_number() over(), rank() over(), dense_rank() over(), percent_rank() over(), lead(c_int, 2, c_int) over(), lag(c_float, 2, c_float) over() from cbo_t1) cbo_t1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from (select count(c_int) over(), sum(c_float) over(), max(c_int) over(), min(c_int) over(), row_number() over(), rank() over(), dense_rank() over(), percent_rank() over(), lead(c_int, 2, c_int) over(), lag(c_float, 2, c_float) over() from cbo_t1) cbo_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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: 0 (type: int) + sort order: + + Map-reduce partition columns: 0 (type: int) + Statistics: Num rows: 20 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: c_int (type: int), c_float (type: float) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col2 (type: int), VALUE._col3 (type: float) + outputColumnNames: _col2, _col3 + Statistics: Num rows: 20 Data size: 9192 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: int, _col3: float + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: 0 ASC NULLS FIRST + partition by: 0 + raw input shape: + window functions: + window function definition + alias: count_window_0 + arguments: _col2 + name: count + window function: GenericUDAFCountEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: sum_window_1 + arguments: _col3 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: max_window_2 + arguments: _col2 + name: max + window function: GenericUDAFMaxEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: min_window_3 + arguments: _col2 + name: min + window function: GenericUDAFMinEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: row_number_window_4 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: rank_window_5 + arguments: 0 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_6 + arguments: 0 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: percent_rank_window_7 + arguments: 0 + name: percent_rank + window function: GenericUDAFPercentRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: lead_window_8 + arguments: _col2, 2, _col2 + name: lead + window function: GenericUDAFLeadEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: lag_window_9 + arguments: _col3, 2, _col3 + name: lag + window function: GenericUDAFLagEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 20 Data size: 9192 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: count_window_0 (type: bigint), sum_window_1 (type: double), max_window_2 (type: int), min_window_3 (type: int), row_number_window_4 (type: int), rank_window_5 (type: int), dense_rank_window_6 (type: int), percent_rank_window_7 (type: double), lead_window_8 (type: int), lag_window_9 (type: float) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Statistics: Num rows: 20 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 20 Data size: 1040 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select i, a, h, b, c, d, e, f, g, a as x, a +1 as y from (select max(c_int) over (partition by key order by value range UNBOUNDED PRECEDING) a, min(c_int) over (partition by key order by value range current row) b, count(c_int) over(partition by key order by value range 1 PRECEDING) c, avg(value) over (partition by key order by value range between unbounded preceding and unbounded following) d, sum(value) over (partition by key order by value range between unbounded preceding and current row) e, avg(c_float) over (partition by key order by value range between 1 preceding and unbounded following) f, sum(c_float) over (partition by key order by value range between 1 preceding and current row) g, max(c_float) over (partition by key order by value range between 1 preceding and unbounded following) h, min(c_float) over (partition by key order by value range between 1 preceding and 1 following) i from cbo_t1) cbo_t1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select i, a, h, b, c, d, e, f, g, a as x, a +1 as y from (select max(c_int) over (partition by key order by value range UNBOUNDED PRECEDING) a, min(c_int) over (partition by key order by value range current row) b, count(c_int) over(partition by key order by value range 1 PRECEDING) c, avg(value) over (partition by key order by value range between unbounded preceding and unbounded following) d, sum(value) over (partition by key order by value range between unbounded preceding and current row) e, avg(c_float) over (partition by key order by value range between 1 preceding and unbounded following) f, sum(c_float) over (partition by key order by value range between 1 preceding and current row) g, max(c_float) over (partition by key order by value range between 1 preceding and unbounded following) h, min(c_float) over (partition by key order by value range between 1 preceding and 1 following) i from cbo_t1) cbo_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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: cbo_t1 + Statistics: Num rows: 20 Data size: 3382 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: key (type: string), value (type: string) + sort order: ++ + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 20 Data size: 3382 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: c_int (type: int), c_float (type: float) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: float) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 20 Data size: 12422 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: string, _col1: string, _col2: int, _col3: float + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: min_window_0 + arguments: _col3 + name: min + window function: GenericUDAFMinEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: max_window_1 + arguments: _col2 + name: max + window function: GenericUDAFMaxEvaluator + window frame: PRECEDING(MAX)~CURRENT + window function definition + alias: max_window_2 + arguments: _col3 + name: max + window function: GenericUDAFMaxEvaluator + window frame: PRECEDING(1)~FOLLOWING(MAX) + window function definition + alias: min_window_3 + arguments: _col2 + name: min + window function: GenericUDAFMinEvaluator + window frame: CURRENT~CURRENT + window function definition + alias: count_window_4 + arguments: _col2 + name: count + window function: GenericUDAFCountEvaluator + window frame: PRECEDING(1)~CURRENT + window function definition + alias: avg_window_5 + arguments: _col1 + name: avg + window function: GenericUDAFAverageEvaluatorDouble + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + window function definition + alias: sum_window_6 + arguments: _col1 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~CURRENT + window function definition + alias: avg_window_7 + arguments: _col3 + name: avg + window function: GenericUDAFAverageEvaluatorDouble + window frame: PRECEDING(1)~FOLLOWING(MAX) + window function definition + alias: sum_window_8 + arguments: _col3 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(1)~CURRENT + Statistics: Num rows: 20 Data size: 12422 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: min_window_0 (type: float), max_window_1 (type: int), max_window_2 (type: float), min_window_3 (type: int), count_window_4 (type: bigint), avg_window_5 (type: double), sum_window_6 (type: double), avg_window_7 (type: double), sum_window_8 (type: double), max_window_1 (type: int), (max_window_1 + 1) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 20 Data size: 1280 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 20 Data size: 1280 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select *, rank() over(partition by key order by value) as rr from src1 +PREHOOK: type: QUERY +POSTHOOK: query: explain select *, rank() over(partition by key order by value) as rr from src1 +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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src1 + Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: key (type: string), value (type: string) + sort order: ++ + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 25 Data size: 11075 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: string, _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 25 Data size: 11075 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), rank_window_0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 25 Data size: 4475 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 25 Data size: 4475 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (select x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: explain +select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (select x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +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 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: x + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Map 5 + Map Operator Tree: + TableScan + alias: y + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 60 Data size: 5160 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + 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: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col0), sum(_col1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (select x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: explain +select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (select x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +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 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: x + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Map 5 + Map Operator Tree: + TableScan + alias: y + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 60 Data size: 5160 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + 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: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col0), sum(_col1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (select x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: explain +select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (select x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: x + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (GROUP, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: y + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + input vertices: + 0 Map 1 + Statistics: Num rows: 60 Data size: 5160 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Local Work: + Map Reduce Local Work + 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: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 14 Data size: 1316 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col0), sum(_col1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (select x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: explain +select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (select x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +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 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: x + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Map 5 + Map Operator Tree: + TableScan + alias: y + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 205 Data size: 17835 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 205 Data size: 17835 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE + 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: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col0), sum(_col1) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain create table abcd (a int, b int, c int, d int) +PREHOOK: type: CREATETABLE +POSTHOOK: query: explain create table abcd (a int, b int, c int, d int) +POSTHOOK: type: CREATETABLE +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: a int, b int, c int, d int + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.abcd + +PREHOOK: query: create table abcd (a int, b int, c int, d int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@abcd +POSTHOOK: query: create table abcd (a int, b int, c int, d int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@abcd +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in4.txt' INTO TABLE abcd +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@abcd +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in4.txt' INTO TABLE abcd +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@abcd +PREHOOK: query: explain select a, count(distinct b), count(distinct c), sum(d) from abcd group by a +PREHOOK: type: QUERY +POSTHOOK: query: explain select a, count(distinct b), count(distinct c), sum(d) from abcd group by 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 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: abcd + Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: a (type: int), b (type: int), c (type: int), d (type: int) + outputColumnNames: a, b, c, d + Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(DISTINCT b), count(DISTINCT c), sum(d) + keys: a (type: int), b (type: int), c (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + sort order: +++ + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE + value expressions: _col5 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0), count(DISTINCT KEY._col1:1._col0), sum(VALUE._col2) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 2 Data size: 39 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 39 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select a, count(distinct b), count(distinct c), sum(d) from abcd group by a +PREHOOK: type: QUERY +POSTHOOK: query: explain select a, count(distinct b), count(distinct c), sum(d) from abcd group by 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 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: abcd + Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: a (type: int), b (type: int), c (type: int), d (type: int) + outputColumnNames: a, b, c, d + Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: a (type: int), b (type: int), c (type: int) + sort order: +++ + Map-reduce partition columns: a (type: int) + Statistics: Num rows: 4 Data size: 78 Basic stats: COMPLETE Column stats: NONE + value expressions: d (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0), count(DISTINCT KEY._col1:1._col0), sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: complete + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 2 Data size: 39 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 39 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain create table src_rc_merge_test(key int, value string) stored as rcfile +PREHOOK: type: CREATETABLE +POSTHOOK: query: explain create table src_rc_merge_test(key int, value string) stored as rcfile +POSTHOOK: type: CREATETABLE +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: key int, value string + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde name: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.src_rc_merge_test + +PREHOOK: query: create table src_rc_merge_test(key int, value string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_rc_merge_test +POSTHOOK: query: create table src_rc_merge_test(key int, value string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_rc_merge_test +PREHOOK: query: load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@src_rc_merge_test +POSTHOOK: query: load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@src_rc_merge_test +PREHOOK: query: explain create table tgt_rc_merge_test(key int, value string) stored as rcfile +PREHOOK: type: CREATETABLE +POSTHOOK: query: explain create table tgt_rc_merge_test(key int, value string) stored as rcfile +POSTHOOK: type: CREATETABLE +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: key int, value string + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde name: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.tgt_rc_merge_test + +PREHOOK: query: create table tgt_rc_merge_test(key int, value string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tgt_rc_merge_test +POSTHOOK: query: create table tgt_rc_merge_test(key int, value string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tgt_rc_merge_test +PREHOOK: query: insert into table tgt_rc_merge_test select * from src_rc_merge_test +PREHOOK: type: QUERY +PREHOOK: Input: default@src_rc_merge_test +PREHOOK: Output: default@tgt_rc_merge_test +POSTHOOK: query: insert into table tgt_rc_merge_test select * from src_rc_merge_test +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_rc_merge_test +POSTHOOK: Output: default@tgt_rc_merge_test +POSTHOOK: Lineage: tgt_rc_merge_test.key SIMPLE [(src_rc_merge_test)src_rc_merge_test.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: tgt_rc_merge_test.value SIMPLE [(src_rc_merge_test)src_rc_merge_test.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: show table extended like `tgt_rc_merge_test` +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like `tgt_rc_merge_test` +POSTHOOK: type: SHOW_TABLESTATUS +tableName:tgt_rc_merge_test +#### A masked pattern was here #### +inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat +outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat +columns:struct columns { i32 key, string value} +partitioned:false +partitionColumns: +totalNumberFiles:1 +totalFileSize:171 +maxFileSize:171 +minFileSize:171 +#### A masked pattern was here #### + +PREHOOK: query: explain select count(1) from tgt_rc_merge_test +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(1) from tgt_rc_merge_test +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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tgt_rc_merge_test + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(1) + mode: complete + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select sum(hash(key)), sum(hash(value)) from tgt_rc_merge_test +PREHOOK: type: QUERY +POSTHOOK: query: explain select sum(hash(key)), sum(hash(value)) from tgt_rc_merge_test +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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tgt_rc_merge_test + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(key) (type: int), hash(value) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: alter table tgt_rc_merge_test concatenate +PREHOOK: type: ALTER_TABLE_MERGE +PREHOOK: Input: default@tgt_rc_merge_test +PREHOOK: Output: default@tgt_rc_merge_test +POSTHOOK: query: alter table tgt_rc_merge_test concatenate +POSTHOOK: type: ALTER_TABLE_MERGE +POSTHOOK: Input: default@tgt_rc_merge_test +POSTHOOK: Output: default@tgt_rc_merge_test +PREHOOK: query: show table extended like `tgt_rc_merge_test` +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like `tgt_rc_merge_test` +POSTHOOK: type: SHOW_TABLESTATUS +tableName:tgt_rc_merge_test +#### A masked pattern was here #### +inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat +outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat +columns:struct columns { i32 key, string value} +partitioned:false +partitionColumns: +totalNumberFiles:1 +totalFileSize:171 +maxFileSize:171 +minFileSize:171 +#### A masked pattern was here #### + +PREHOOK: query: explain select count(1) from tgt_rc_merge_test +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(1) from tgt_rc_merge_test +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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tgt_rc_merge_test + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(1) + mode: complete + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select sum(hash(key)), sum(hash(value)) from tgt_rc_merge_test +PREHOOK: type: QUERY +POSTHOOK: query: explain select sum(hash(key)), sum(hash(value)) from tgt_rc_merge_test +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) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tgt_rc_merge_test + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(key) (type: int), hash(value) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 5 Data size: 32 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: drop table src_rc_merge_test +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@src_rc_merge_test +PREHOOK: Output: default@src_rc_merge_test +POSTHOOK: query: drop table src_rc_merge_test +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@src_rc_merge_test +POSTHOOK: Output: default@src_rc_merge_test +PREHOOK: query: drop table tgt_rc_merge_test +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tgt_rc_merge_test +PREHOOK: Output: default@tgt_rc_merge_test +POSTHOOK: query: drop table tgt_rc_merge_test +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tgt_rc_merge_test +POSTHOOK: Output: default@tgt_rc_merge_test +PREHOOK: query: explain select src.key from src cross join src src2 +PREHOOK: type: QUERY +POSTHOOK: query: explain select src.key from src cross join src src2 +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 3 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: src2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0 + Statistics: Num rows: 250000 Data size: 21750000 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 250000 Data size: 21750000 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain create table nzhang_Tmp(a int, b string) +PREHOOK: type: CREATETABLE +POSTHOOK: query: explain create table nzhang_Tmp(a int, b string) +POSTHOOK: type: CREATETABLE +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: a int, b string + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.nzhang_Tmp + +PREHOOK: query: create table nzhang_Tmp(a int, b string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@nzhang_Tmp +POSTHOOK: query: create table nzhang_Tmp(a int, b string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@nzhang_Tmp +PREHOOK: query: explain create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10 +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: explain create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-0 + Stage-2 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.nzhang_CTAS1 + + Stage: Stage-0 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-3 + Create Table Operator: + Create Table + columns: k string, value string + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.nzhang_CTAS1 + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@nzhang_CTAS1 +POSTHOOK: query: create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@nzhang_CTAS1 +POSTHOOK: Lineage: nzhang_ctas1.k SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: nzhang_ctas1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: explain create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10 +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: explain create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-0 + Stage-2 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: (key / 2) (type: double), concat(value, '_con') (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: double), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: double), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.nzhang_ctas3 + + Stage: Stage-0 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-3 + Create Table Operator: + Create Table + columns: half_key double, conb string + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde name: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.nzhang_ctas3 + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@nzhang_ctas3 +POSTHOOK: query: create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@nzhang_ctas3 +POSTHOOK: Lineage: nzhang_ctas3.conb EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: nzhang_ctas3.half_key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2 +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2 +POSTHOOK: type: CREATETABLE_AS_SELECT +STAGE DEPENDENCIES: + +STAGE PLANS: +PREHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2 +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2 +POSTHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: query: explain create temporary table acid_dtt(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +POSTHOOK: query: explain create temporary table acid_dtt(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + bucket columns: a + columns: a int, b varchar(128) + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +#### A masked pattern was here #### + # buckets: 2 + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde name: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.acid_dtt + table properties: + transactional true + isTemporary: true + +PREHOOK: query: create temporary table acid_dtt(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@acid_dtt +POSTHOOK: query: create temporary table acid_dtt(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@acid_dtt +PREHOOK: query: explain +select src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (select * FROM src WHERE src.key < 10) src1 + JOIN + (select * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +PREHOOK: type: QUERY +POSTHOOK: query: explain +select src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (select * FROM src WHERE src.key < 10) src1 + JOIN + (select * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +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 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: string) + 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: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + sort order: ++++ + Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: CREATE TABLE myinput1(key int, value int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@myinput1 +POSTHOOK: query: CREATE TABLE myinput1(key int, value int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@myinput1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in8.txt' INTO TABLE myinput1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@myinput1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in8.txt' INTO TABLE myinput1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@myinput1 +PREHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value +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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: value (type: int) + sort order: + + Map-reduce partition columns: value (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: key (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 value (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value join myinput1 c on a.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value join myinput1 c on a.key=c.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 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: value is not null (type: boolean) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: value (type: int) + sort order: + + Map-reduce partition columns: value (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: key (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 key (type: int) + 1 value (type: int) + 2 key (type: int) + outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 + Statistics: Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 6 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value join myinput1 c on a.key<=>c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value join myinput1 c on a.key<=>c.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 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: value (type: int) + sort order: + + Map-reduce partition columns: value (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: key (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + keys: + 0 key (type: int) + 1 value (type: int) + 2 key (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 + Statistics: Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 6 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.value=b.key join myinput1 c on a.key<=>c.key AND a.value=c.value +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.value=b.key join myinput1 c on a.key<=>c.key AND a.value=c.value +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 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: value is not null (type: boolean) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int), value (type: int) + sort order: ++ + Map-reduce partition columns: key (type: int), value (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: value (type: int), key (type: int) + sort order: ++ + Map-reduce partition columns: value (type: int), key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: value is not null (type: boolean) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int), value (type: int) + sort order: ++ + Map-reduce partition columns: key (type: int), value (type: int) + Statistics: Num rows: 3 Data size: 26 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 key (type: int), value (type: int) + 1 value (type: int), key (type: int) + 2 key (type: int), value (type: int) + nullSafes: [true, false] + outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 + Statistics: Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 6 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.value<=>b.key join myinput1 c on a.key<=>c.key AND a.value<=>c.value +PREHOOK: type: QUERY +POSTHOOK: query: explain select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.value<=>b.key join myinput1 c on a.key<=>c.key AND a.value<=>c.value +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 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int), value (type: int) + sort order: ++ + Map-reduce partition columns: key (type: int), value (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: value (type: int), key (type: int) + sort order: ++ + Map-reduce partition columns: value (type: int), key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int), value (type: int) + sort order: ++ + Map-reduce partition columns: key (type: int), value (type: int) + Statistics: Num rows: 3 Data size: 26 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 key (type: int), value (type: int) + 1 value (type: int), key (type: int) + 2 key (type: int), value (type: int) + nullSafes: [true, true] + outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 + Statistics: Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 57 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 6 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key<=>b.value +PREHOOK: type: QUERY +POSTHOOK: query: explain select * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key<=>b.value +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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: value (type: int) + sort order: + + Map-reduce partition columns: value (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: key (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 key (type: int) + 1 value (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key<=>b.value +PREHOOK: type: QUERY +POSTHOOK: query: explain select * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key<=>b.value +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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: value (type: int) + sort order: + + Map-reduce partition columns: value (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: key (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + keys: + 0 key (type: int) + 1 value (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select * FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.key<=>b.value +PREHOOK: type: QUERY +POSTHOOK: query: explain select * FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.key<=>b.value +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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: value (type: int) + sort order: + + Map-reduce partition columns: value (type: int) + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + value expressions: key (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + keys: + 0 key (type: int) + 1 value (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key<=>b.value +PREHOOK: type: QUERY +POSTHOOK: query: explain select /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key<=>b.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: int) + 1 value (type: int) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 3 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 value (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + input vertices: + 1 Map 2 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 3 Data size: 28 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 3 Data size: 28 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: CREATE TABLE smb_input(key int, value int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_input +POSTHOOK: query: CREATE TABLE smb_input(key int, value int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_input +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in4.txt' into table smb_input +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@smb_input +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in4.txt' into table smb_input +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@smb_input +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in5.txt' into table smb_input +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@smb_input +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in5.txt' into table smb_input +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@smb_input +PREHOOK: query: CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_input1 +POSTHOOK: query: CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_input1 +PREHOOK: query: CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_input2 +POSTHOOK: query: CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_input2 +PREHOOK: query: from smb_input +insert overwrite table smb_input1 select * +insert overwrite table smb_input2 select * +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_input +PREHOOK: Output: default@smb_input1 +PREHOOK: Output: default@smb_input2 +POSTHOOK: query: from smb_input +insert overwrite table smb_input1 select * +insert overwrite table smb_input2 select * +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_input +POSTHOOK: Output: default@smb_input1 +POSTHOOK: Output: default@smb_input2 +POSTHOOK: Lineage: smb_input1.key SIMPLE [(smb_input)smb_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_input1.value SIMPLE [(smb_input)smb_input.FieldSchema(name:value, type:int, comment:null), ] +POSTHOOK: Lineage: smb_input2.key SIMPLE [(smb_input)smb_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_input2.value SIMPLE [(smb_input)smb_input.FieldSchema(name:value, type:int, comment:null), ] +PREHOOK: query: analyze table smb_input1 compute statistics +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_input1 +PREHOOK: Output: default@smb_input1 +POSTHOOK: query: analyze table smb_input1 compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_input1 +POSTHOOK: Output: default@smb_input1 +PREHOOK: query: explain select /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.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 +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 26 Data size: 190 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 28 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.key AND a.value <=> b.value +PREHOOK: type: QUERY +POSTHOOK: query: explain select /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.key AND a.value <=> b.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 26 Data size: 190 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: int), value (type: int) + 1 key (type: int), value (type: int) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 26 Data size: 190 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), value (type: int) + 1 key (type: int), value (type: int) + nullSafes: [true, true] + outputColumnNames: _col0, _col1, _col5, _col6 + input vertices: + 0 Map 1 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 28 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input1 b ON a.key <=> b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input1 b ON a.key <=> b.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 +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 26 Data size: 190 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 28 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.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 +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 26 Data size: 190 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 28 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input1 b ON a.key <=> b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input1 b ON a.key <=> b.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 +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 26 Data size: 190 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + nullSafes: [true] + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 28 Data size: 209 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 28 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: drop table sales +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table sales +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table things +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table things +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE sales (name STRING, id INT) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@sales +POSTHOOK: query: CREATE TABLE sales (name STRING, id INT) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@sales +PREHOOK: query: CREATE TABLE things (id INT, name STRING) partitioned by (ds string) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@things +POSTHOOK: query: CREATE TABLE things (id INT, name STRING) partitioned by (ds string) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@things +PREHOOK: query: load data local inpath '../../data/files/sales.txt' INTO TABLE sales +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@sales +POSTHOOK: query: load data local inpath '../../data/files/sales.txt' INTO TABLE sales +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@sales +PREHOOK: query: load data local inpath '../../data/files/things.txt' INTO TABLE things partition(ds='2011-10-23') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@things +POSTHOOK: query: load data local inpath '../../data/files/things.txt' INTO TABLE things partition(ds='2011-10-23') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@things +POSTHOOK: Output: default@things@ds=2011-10-23 +PREHOOK: query: load data local inpath '../../data/files/things2.txt' INTO TABLE things partition(ds='2011-10-24') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@things +POSTHOOK: query: load data local inpath '../../data/files/things2.txt' INTO TABLE things partition(ds='2011-10-24') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@things +POSTHOOK: Output: default@things@ds=2011-10-24 +PREHOOK: query: explain select name,id FROM sales LEFT SEMI JOIN things ON (sales.id = things.id) +PREHOOK: type: QUERY +POSTHOOK: query: explain select name,id FROM sales LEFT SEMI JOIN things ON (sales.id = things.id) +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 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: sales + Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: id is not null (type: boolean) + Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: name (type: string), id (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 13 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: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: things + Statistics: Num rows: 2 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: id is not null (type: boolean) + Statistics: Num rows: 2 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: id (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 28 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: 28 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: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 30 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 2 Data size: 30 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: drop table sales +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@sales +PREHOOK: Output: default@sales +POSTHOOK: query: drop table sales +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@sales +POSTHOOK: Output: default@sales +PREHOOK: query: drop table things +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@things +PREHOOK: Output: default@things +POSTHOOK: query: drop table things +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@things +POSTHOOK: Output: default@things +PREHOOK: query: explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450' +PREHOOK: type: QUERY +POSTHOOK: query: explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: src1 + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (value > 'val_450') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((value > 'val_450') and key is not null) (type: boolean) + Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + input vertices: + 1 Map 2 + Statistics: Num rows: 732 Data size: 7782 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + input vertices: + 1 Map 3 + Statistics: Num rows: 805 Data size: 8560 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 805 Data size: 8560 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450' +PREHOOK: type: QUERY +POSTHOOK: query: explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: src1 + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (value > 'val_450') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((value > 'val_450') and key is not null) (type: boolean) + Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + input vertices: + 1 Map 2 + Statistics: Num rows: 732 Data size: 7782 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + input vertices: + 1 Map 3 + Statistics: Num rows: 805 Data size: 8560 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 805 Data size: 8560 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part + partition by p_mfgr + order by p_name + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part + partition by p_mfgr + order by 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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int), p_retailprice (type: double) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col7 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), rank_window_0 (type: int), dense_rank_window_1 (type: int), sum_window_2 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop (on (select p1.* from part p1 join part p2 on p1.p_partkey = p2.p_partkey) j +distribute by j.p_mfgr +sort by j.p_name) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop (on (select p1.* from part p1 join part p2 on p1.p_partkey = p2.p_partkey) j +distribute by j.p_mfgr +sort by j.p_name) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: p2 + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Spark HashTable Sink Operator + keys: + 0 p_partkey (type: int) + 1 p_partkey (type: int) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: p1 + Statistics: Num rows: 26 Data size: 5902 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 5902 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 p_partkey (type: int) + 1 p_partkey (type: int) + outputColumnNames: _col1, _col2, _col5 + input vertices: + 1 Map 4 + Statistics: Num rows: 29 Data size: 6467 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 29 Data size: 6467 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int) + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 29 Data size: 6467 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: j + output shape: _col1: string, _col2: string, _col5: int + type: SUBQUERY + Partition table definition + input alias: ptf_1 + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Statistics: Num rows: 29 Data size: 6467 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 29 Data size: 6467 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 29 Data size: 6467 Basic stats: COMPLETE Column stats: COMPLETE + 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: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: lag_window_0 + arguments: _col5, 1, _col5 + name: lag + window function: GenericUDAFLagEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 29 Data size: 6467 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), (_col5 - lag_window_0) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 29 Data size: 6583 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 29 Data size: 6583 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part + partition by p_mfgr + order by p_name + ) abc +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part + partition by p_mfgr + order by p_name + ) abc +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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int), p_retailprice (type: double) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: TABLE + Partition table definition + input alias: abc + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col7 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), rank_window_0 (type: int), dense_rank_window_1 (type: int), sum_window_2 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop(on part + partition by p_mfgr + order by 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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: lag_window_2 + arguments: _col5, 1, _col5 + name: lag + window function: GenericUDAFLagEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), rank_window_0 (type: int), dense_rank_window_1 (type: int), _col5 (type: int), (_col5 - lag_window_2) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +group by p_mfgr, p_name, p_size +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +group by p_mfgr, p_name, 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 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (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: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int) + sort order: +++ + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: int) + mode: partial1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int) + sort order: +++ + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 4 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: int) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: string, _col1: string, _col2: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: lag_window_2 + arguments: _col2, 1, _col2 + name: lag + window function: GenericUDAFLagEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), rank_window_0 (type: int), dense_rank_window_1 (type: int), _col2 (type: int), (_col2 - lag_window_2) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select abc.* +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +PREHOOK: type: QUERY +POSTHOOK: query: explain +select abc.* +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: p1 + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 p_partkey (type: int) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) + Reducer 2 + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (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: 26 Data size: 23062 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string + type: TABLE + Partition table definition + input alias: abc + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 23062 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 26 Data size: 23062 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 p_partkey (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + input vertices: + 1 Map 3 + Statistics: Num rows: 29 Data size: 17951 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 29 Data size: 17951 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name, p_size desc) as r +from noopwithmap(on part +partition by p_mfgr +order by p_name, p_size desc) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name, p_size desc) as r +from noopwithmap(on part +partition by p_mfgr +order by p_name, p_size desc) +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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: p_name: string, p_mfgr: string, p_size: int + type: TABLE + Partition table definition + input alias: ptf_1 + name: noopwithmap + order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST + output shape: p_name: string, p_mfgr: string, p_size: int + partition by: p_mfgr + raw input shape: + transforms raw input: true + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Map-side function: true + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string), p_size (type: int) + sort order: ++- + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey2 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int + type: TABLE + Partition table definition + input alias: ptf_1 + name: noopwithmap + order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + transforms raw input: true + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int) + sort order: ++- + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), KEY.reducesinkkey2 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1, _col5 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), rank_window_0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 26 Data size: 5902 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 5902 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noopwithmap(on part + partition by p_mfgr + order by p_name) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noopwithmap(on part + partition by p_mfgr + order by 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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double + type: TABLE + Partition table definition + input alias: ptf_1 + name: noopwithmap + order by: p_name ASC NULLS FIRST + output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double + partition by: p_mfgr + raw input shape: + transforms raw input: true + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + Map-side function: true + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int), p_retailprice (type: double) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: TABLE + Partition table definition + input alias: ptf_1 + name: noopwithmap + order by: _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + transforms raw input: true + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col7 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), rank_window_0 (type: int), dense_rank_window_1 (type: int), sum_window_2 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part +partition by p_mfgr +order by p_name) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part +partition by p_mfgr +order by 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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int), p_retailprice (type: double) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col7 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), rank_window_0 (type: int), dense_rank_window_1 (type: int), sum_window_2 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on noopwithmap(on noop(on part +partition by p_mfgr +order by p_mfgr DESC, p_name +))) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on noopwithmap(on noop(on part +partition by p_mfgr +order by p_mfgr DESC, 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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: -+ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int), p_retailprice (type: double) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: PTFCOMPONENT + Partition table definition + input alias: ptf_1 + name: noopwithmap + order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + transforms raw input: true + Partition table definition + input alias: ptf_2 + name: noop + order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Map-side function: true + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: -+ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: PTFCOMPONENT + Partition table definition + input alias: ptf_1 + name: noopwithmap + order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + transforms raw input: true + Partition table definition + input alias: ptf_2 + name: noop + order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col7 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), rank_window_0 (type: int), dense_rank_window_1 (type: int), sum_window_2 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, +sub1.cd, sub1.s1 +from (select p_mfgr, p_name, +count(p_size) over (partition by p_mfgr order by p_name) as cd, +p_retailprice, +sum(p_retailprice) over w1 as s1 +from noop(on part +partition by p_mfgr +order by p_name) +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +) sub1 +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, +sub1.cd, sub1.s1 +from (select p_mfgr, p_name, +count(p_size) over (partition by p_mfgr order by p_name) as cd, +p_retailprice, +sum(p_retailprice) over w1 as s1 +from noop(on part +partition by p_mfgr +order by p_name) +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +) sub1 +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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int), p_retailprice (type: double) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: count_window_0 + arguments: _col5 + name: count + window function: GenericUDAFCountEvaluator + window frame: PRECEDING(MAX)~CURRENT + window function definition + alias: sum_window_1 + arguments: _col7 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(2)~FOLLOWING(2) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), count_window_0 (type: bigint), sum_window_1 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select abc.p_mfgr, abc.p_name, +rank() over (distribute by abc.p_mfgr sort by abc.p_name) as r, +dense_rank() over (distribute by abc.p_mfgr sort by abc.p_name) as dr, +count(abc.p_name) over (distribute by abc.p_mfgr sort by abc.p_name) as cd, +abc.p_retailprice, sum(abc.p_retailprice) over (distribute by abc.p_mfgr sort by abc.p_name rows between unbounded preceding and current row) as s1, +abc.p_size, abc.p_size - lag(abc.p_size,1,abc.p_size) over (distribute by abc.p_mfgr sort by abc.p_name) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +PREHOOK: type: QUERY +POSTHOOK: query: explain +select abc.p_mfgr, abc.p_name, +rank() over (distribute by abc.p_mfgr sort by abc.p_name) as r, +dense_rank() over (distribute by abc.p_mfgr sort by abc.p_name) as dr, +count(abc.p_name) over (distribute by abc.p_mfgr sort by abc.p_name) as cd, +abc.p_retailprice, sum(abc.p_retailprice) over (distribute by abc.p_mfgr sort by abc.p_name rows between unbounded preceding and current row) as s1, +abc.p_size, abc.p_size - lag(abc.p_size,1,abc.p_size) over (distribute by abc.p_mfgr sort by abc.p_name) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: p1 + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: p_partkey is not null (type: boolean) + Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 p_partkey (type: int) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_partkey (type: int), p_size (type: int), p_retailprice (type: double) + Reducer 2 + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col0, _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double + type: TABLE + Partition table definition + input alias: abc + name: noop + order by: _col1 ASC NULLS FIRST + output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 p_partkey (type: int) + outputColumnNames: _col1, _col2, _col5, _col7 + input vertices: + 1 Map 4 + Statistics: Num rows: 29 Data size: 6699 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 29 Data size: 6699 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 29 Data size: 22243 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: count_window_2 + arguments: _col1 + name: count + window function: GenericUDAFCountEvaluator + window frame: PRECEDING(MAX)~CURRENT + window function definition + alias: sum_window_3 + arguments: _col7 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~CURRENT + window function definition + alias: lag_window_4 + arguments: _col5, 1, _col5 + name: lag + window function: GenericUDAFLagEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 29 Data size: 22243 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), rank_window_0 (type: int), dense_rank_window_1 (type: int), count_window_2 (type: bigint), _col7 (type: double), sum_window_3 (type: double), _col5 (type: int), (_col5 - lag_window_4) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 29 Data size: 7511 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 29 Data size: 7511 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand +PREHOOK: type: CREATEVIEW +POSTHOOK: query: explain create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand +POSTHOOK: type: CREATEVIEW +STAGE DEPENDENCIES: + Stage-1 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Create View Operator: + Create View + if not exists: true + or replace: false + columns: p_mfgr string, p_brand string, s double + expanded text: select `part`.`p_mfgr`, `part`.`p_brand`, +sum(`part`.`p_retailprice`) as `s` +from `default`.`part` +group by `part`.`p_mfgr`, `part`.`p_brand` + name: default.mfgr_price_view + original text: select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand + rewrite enabled: false + +PREHOOK: query: CREATE TABLE part_4( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@part_4 +POSTHOOK: query: CREATE TABLE part_4( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@part_4 +PREHOOK: query: CREATE TABLE part_5( +p_mfgr STRING, +p_name STRING, +p_size INT, +s2 INT, +r INT, +dr INT, +cud DOUBLE, +fv1 INT) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@part_5 +POSTHOOK: query: CREATE TABLE part_5( +p_mfgr STRING, +p_name STRING, +p_size INT, +s2 INT, +r INT, +dr INT, +cud DOUBLE, +fv1 INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@part_5 +PREHOOK: query: explain +from noop(on part +partition by p_mfgr +order by p_name) +INSERT OVERWRITE TABLE part_4 select p_mfgr, p_name, p_size, +rank() over (distribute by p_mfgr sort by p_name) as r, +dense_rank() over (distribute by p_mfgr sort by p_name) as dr, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s +INSERT OVERWRITE TABLE part_5 select p_mfgr,p_name, p_size, +round(sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row),1) as s2, +rank() over (distribute by p_mfgr sort by p_mfgr, p_name) as r, +dense_rank() over (distribute by p_mfgr sort by p_mfgr, p_name) as dr, +cume_dist() over (distribute by p_mfgr sort by p_mfgr, p_name) as cud, +first_value(p_size, true) over w1 as fv1 +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +POSTHOOK: query: explain +from noop(on part +partition by p_mfgr +order by p_name) +INSERT OVERWRITE TABLE part_4 select p_mfgr, p_name, p_size, +rank() over (distribute by p_mfgr sort by p_name) as r, +dense_rank() over (distribute by p_mfgr sort by p_name) as dr, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s +INSERT OVERWRITE TABLE part_5 select p_mfgr,p_name, p_size, +round(sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row),1) as s2, +rank() over (distribute by p_mfgr sort by p_mfgr, p_name) as r, +dense_rank() over (distribute by p_mfgr sort by p_mfgr, p_name) as dr, +cume_dist() over (distribute by p_mfgr sort by p_mfgr, p_name) as cud, +first_value(p_size, true) over w1 as fv1 +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + Stage-1 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 3 <- Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 1 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 6006 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int), p_retailprice (type: double) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int, _col7: double + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col7 + name: sum + window function: GenericUDAFSumDouble + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), rank_window_0 (type: int), dense_rank_window_1 (type: int), sum_window_2 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.part_4 + Reducer 4 + 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: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: sum_window_0 + arguments: _col5 + name: sum + window function: GenericUDAFSumLong + window frame: PRECEDING(5)~CURRENT + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: sum_window_0 (type: bigint), _col1 (type: string), _col2 (type: string), _col5 (type: int) + outputColumnNames: sum_window_0, _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: sum_window_0 (type: bigint), _col5 (type: int) + Reducer 5 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: bigint), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col4 (type: int) + outputColumnNames: _col0, _col2, _col3, _col6 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: bigint, _col2: string, _col3: string, _col6: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST + partition by: _col3 + raw input shape: + window functions: + window function definition + alias: rank_window_1 + arguments: _col3, _col2 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_2 + arguments: _col3, _col2 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: cume_dist_window_3 + arguments: _col3, _col2 + name: cume_dist + window function: GenericUDAFCumeDistEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: first_value_window_4 + arguments: _col6, true + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(2)~FOLLOWING(2) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: string), _col2 (type: string), _col6 (type: int), UDFToInteger(round(_col0, 1)) (type: int), rank_window_1 (type: int), dense_rank_window_2 (type: int), cume_dist_window_3 (type: double), first_value_window_4 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 26 Data size: 6422 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6422 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.part_5 + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int), _col7 (type: double) + Reducer 7 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double) + outputColumnNames: _col1, _col2, _col5, _col7 + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col5 (type: int) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12974 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.part_4 + + Stage: Stage-3 + Stats-Aggr Operator + + Stage: Stage-1 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.part_5 + + Stage: Stage-4 + Stats-Aggr Operator + +PREHOOK: query: explain +select p_mfgr, p_name, +rank() over (partition by p_mfgr,p_name) as r, +dense_rank() over (partition by p_mfgr,p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr,p_name rows between unbounded preceding and current row) as s1 +from noop(on + noopwithmap(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr,p_name + order by p_mfgr,p_name) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, +rank() over (partition by p_mfgr,p_name) as r, +dense_rank() over (partition by p_mfgr,p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr,p_name rows between unbounded preceding and current row) as s1 +from noop(on + noopwithmap(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr,p_name + order by p_mfgr,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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string) + sort order: + + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string), p_size (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col4 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col2 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Partition table definition + input alias: ptf_2 + name: noop + order by: _col2 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int + type: PTFCOMPONENT + Partition table definition + input alias: ptf_1 + name: noopwithmap + order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2, _col1 + raw input shape: + transforms raw input: true + Partition table definition + input alias: ptf_2 + name: noop + order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2, _col1 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Map-side function: true + 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: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int + type: PTFCOMPONENT + Partition table definition + input alias: ptf_1 + name: noopwithmap + order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2, _col1 + raw input shape: + transforms raw input: true + Partition table definition + input alias: ptf_2 + name: noop + order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2, _col1 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST + partition by: _col2, _col1 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col2, _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col2, _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col5 + name: sum + window function: GenericUDAFSumLong + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), rank_window_0 (type: int), dense_rank_window_1 (type: int), _col5 (type: int), sum_window_2 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr + order by p_mfgr ) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr + order by 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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string) + sort order: + + Map-reduce partition columns: p_mfgr (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string), p_size (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col4 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col2 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Partition table definition + input alias: ptf_2 + name: noop + order by: _col2 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int + type: PTFCOMPONENT + Partition table definition + input alias: ptf_1 + name: noop + order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2, _col1 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col5 (type: int) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col4 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int + type: PTFCOMPONENT + Partition table definition + input alias: ptf_1 + name: noop + order by: _col2 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int) + Reducer 5 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col5 + name: sum + window function: GenericUDAFSumLong + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), rank_window_0 (type: int), dense_rank_window_1 (type: int), _col5 (type: int), sum_window_2 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr order by p_name) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr)) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr order by p_name) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by 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 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: part + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: p_mfgr (type: string), p_name (type: string) + sort order: ++ + Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_size (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: part + output shape: _col1: string, _col2: string, _col5: int + type: TABLE + Partition table definition + input alias: ptf_1 + name: noop + order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2, _col1 + raw input shape: + Partition table definition + input alias: ptf_2 + name: noop + order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2, _col1 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col5 (type: int) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col4 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string, _col2: string, _col5: int + type: PTFCOMPONENT + Partition table definition + input alias: ptf_1 + name: noop + order by: _col2 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Partition table definition + input alias: ptf_2 + name: noop + order by: _col2 ASC NULLS FIRST + output shape: _col1: string, _col2: string, _col5: int + partition by: _col2 + raw input shape: + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: int) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int) + outputColumnNames: _col1, _col2, _col5 + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + 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: _col1 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: rank_window_0 + arguments: _col1 + name: rank + window function: GenericUDAFRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: dense_rank_window_1 + arguments: _col1 + name: dense_rank + window function: GenericUDAFDenseRankEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + window function definition + alias: sum_window_2 + arguments: _col5 + name: sum + window function: GenericUDAFSumLong + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 26 Data size: 12766 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string), _col1 (type: string), rank_window_0 (type: int), dense_rank_window_1 (type: int), _col5 (type: int), sum_window_2 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 26 Data size: 6214 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select distinct src.* from src +PREHOOK: type: QUERY +POSTHOOK: query: explain select distinct src.* 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 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: key, value + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: key (type: string), value (type: string) + sort order: ++ + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: partial1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: final + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select explode(array('a', 'b')) +PREHOOK: type: QUERY +POSTHOOK: query: explain select explode(array('a', 'b')) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + TableScan + alias: _dummy_table + Row Limit Per Split: 1 + Select Operator + expressions: array('a','b') (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + ListSink + +PREHOOK: query: CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T1 +POSTHOOK: query: CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1 +PREHOOK: query: CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T2 +POSTHOOK: query: CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T2 +PREHOOK: query: CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T3 +POSTHOOK: query: CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T3 +PREHOOK: query: CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T4 +POSTHOOK: query: CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T4 +PREHOOK: query: CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest_j1 +POSTHOOK: query: CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest_j1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t2 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t2 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t3 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t3 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t4 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t4 +PREHOOK: query: explain +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 select src1.key, src2.value +PREHOOK: type: QUERY +POSTHOOK: query: explain +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 select src1.key, src2.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-4 depends on stages: Stage-1 , consists of Stage-5, Stage-0 + Stage-5 + Stage-3 depends on stages: Stage-5 + Stage-0 depends on stages: Stage-3 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src1 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Map 3 + Map Operator Tree: + TableScan + alias: src2 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + handleSkewJoin: true + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 1219 Data size: 216982 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1219 Data size: 115805 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 1219 Data size: 115805 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_j1 + + Stage: Stage-4 + Conditional Operator + + Stage: Stage-5 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + Spark HashTable Sink Operator + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + outputColumnNames: _col0, _col2 + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1219 Data size: 115805 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 1219 Data size: 115805 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_j1 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_j1 + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 select src1.key, src2.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dest_j1 +POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 select src1.key, src2.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dest_j1 +POSTHOOK: Lineage: dest_j1.key EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest_j1.value SIMPLE [(src)src2.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: explain +select /*+ STREAMTABLE(a) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+ STREAMTABLE(a) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + 3 _col0 (type: string) + Local Work: + Map Reduce Local Work + Map 3 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + 3 _col0 (type: string) + Local Work: + Map Reduce Local Work + Map 4 + Map Operator Tree: + TableScan + alias: d + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + 3 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + Inner Join 2 to 3 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + 3 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + input vertices: + 1 Map 2 + 2 Map 3 + 3 Map 4 + Statistics: Num rows: 3 Data size: 99 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 3 Data size: 99 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select /*+ STREAMTABLE(a,c) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+ STREAMTABLE(a,c) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + 3 _col0 (type: string) + Local Work: + Map Reduce Local Work + Map 3 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + 3 _col0 (type: string) + Local Work: + Map Reduce Local Work + Map 4 + Map Operator Tree: + TableScan + alias: d + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + 3 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + Inner Join 2 to 3 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + 2 _col0 (type: string) + 3 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + input vertices: + 1 Map 2 + 2 Map 3 + 3 Map 4 + Statistics: Num rows: 3 Data size: 99 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 3 Data size: 99 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain FROM T1 a JOIN src c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +PREHOOK: type: QUERY +POSTHOOK: query: explain FROM T1 a JOIN src c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 UDFToDouble(_col0) (type: double) + 1 (UDFToDouble(_col0) + 1.0) (type: double) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (GROUP, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 (UDFToDouble(_col0) + 1.0) (type: double) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 0 Map 1 + Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Local Work: + Map Reduce Local Work + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2) + mode: partial1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: FROM T1 a JOIN src c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: FROM T1 a JOIN src c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +198 6274 194 +PREHOOK: query: explain +select * FROM +(select src.* FROM src) x +JOIN +(select src.* FROM src) Y +ON (x.key = Y.key) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * FROM +(select src.* FROM src) x +JOIN +(select src.* FROM src) Y +ON (x.key = Y.key) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-3 depends on stages: Stage-1 , consists of Stage-4 + Stage-4 + Stage-2 depends on stages: Stage-4 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + handleSkewJoin: true + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1219 Data size: 433964 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: true + Statistics: Num rows: 1219 Data size: 433964 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-3 + Conditional Operator + + Stage: Stage-4 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + Spark HashTable Sink Operator + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: true + Statistics: Num rows: 1219 Data size: 433964 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select /*+ mapjoin(k)*/ sum(hash(k.key)), sum(hash(v.val)) from T1 k join T1 v on k.key=v.val +PREHOOK: type: QUERY +POSTHOOK: query: explain select /*+ mapjoin(k)*/ sum(hash(k.key)), sum(hash(v.val)) from T1 k join T1 v on k.key=v.val +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: v + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: val is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: val (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: k + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + input vertices: + 1 Map 4 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int) + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: partial1 + 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 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: final + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select sum(hash(k.key)), sum(hash(v.val)) from T1 k join T1 v on k.key=v.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select sum(hash(k.key)), sum(hash(v.val)) from T1 k join T1 v on k.key=v.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: v + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: k + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col2 + input vertices: + 1 Map 4 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(_col0) (type: int), hash(_col2) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int) + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: partial1 + 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 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: final + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select count(1) from T1 a join T1 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select count(1) from T1 a join T1 b on a.key = b.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(1) + mode: partial1 + 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: final + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain FROM T1 a LEFT OUTER JOIN T2 c ON c.key+1=a.key select sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +PREHOOK: type: QUERY +POSTHOOK: query: explain FROM T1 a LEFT OUTER JOIN T2 c ON c.key+1=a.key select sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 UDFToDouble(_col0) (type: double) + 1 (UDFToDouble(_col0) + 1.0) (type: double) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 (UDFToDouble(_col0) + 1.0) (type: double) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 1 Map 4 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2) + mode: partial1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain FROM T1 a RIGHT OUTER JOIN T2 c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +PREHOOK: type: QUERY +POSTHOOK: query: explain FROM T1 a RIGHT OUTER JOIN T2 c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 UDFToDouble(_col0) (type: double) + 1 (UDFToDouble(_col0) + 1.0) (type: double) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (GROUP, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Right Outer Join0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 (UDFToDouble(_col0) + 1.0) (type: double) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 0 Map 1 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Local Work: + Map Reduce Local Work + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2) + mode: partial1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain FROM T1 a FULL OUTER JOIN T2 c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)) +PREHOOK: type: QUERY +POSTHOOK: query: explain FROM T1 a FULL OUTER JOIN T2 c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.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 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 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: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: c + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: (UDFToDouble(_col0) + 1.0) (type: double) + sort order: + + Map-reduce partition columns: (UDFToDouble(_col0) + 1.0) (type: double) + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 (UDFToDouble(_col0) + 1.0) (type: double) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2) + mode: partial1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select /*+ mapjoin(v)*/ sum(hash(k.key)), sum(hash(v.val)) from T1 k left outer join T1 v on k.key+1=v.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select /*+ mapjoin(v)*/ sum(hash(k.key)), sum(hash(v.val)) from T1 k left outer join T1 v on k.key+1=v.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: v + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), val (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 (UDFToDouble(_col0) + 1.0) (type: double) + 1 UDFToDouble(_col0) (type: double) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: k + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 (UDFToDouble(_col0) + 1.0) (type: double) + 1 UDFToDouble(_col0) (type: double) + outputColumnNames: _col0, _col2 + input vertices: + 1 Map 4 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hash(_col0) (type: int), hash(_col2) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: int) + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: partial1 + 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 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + mode: final + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: true + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink +