diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 1502d80..327ea95 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -304,6 +304,7 @@ minitez.query.files=bucket_map_join_tez1.q,\ tez_schema_evolution.q,\ tez_union.q,\ tez_union2.q,\ + tez_union_view.q,\ tez_union_decimal.q,\ tez_union_group_by.q,\ tez_smb_main.q,\ diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java index 44ab1bd..6374ec1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java @@ -364,6 +364,17 @@ private boolean validateMapWork(MapWork mapWork, boolean isTez) throws SemanticE addMapWorkRules(opRules, vnp); Dispatcher disp = new DefaultRuleDispatcher(vnp, opRules, null); GraphWalker ogw = new DefaultGraphWalker(disp); + if ((mapWork.getAliasToWork() == null) || (mapWork.getAliasToWork().size() == 0)) { + return false; + } else { + for (Operator op : mapWork.getAliasToWork().values()) { + if (op == null) { + LOG.warn("Map work has invalid aliases to work with. Fail validation!"); + return false; + } + } + } + // iterator the mapper operator tree ArrayList topNodes = new ArrayList(); topNodes.addAll(mapWork.getAliasToWork().values()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java index 05a5841..3217df2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java @@ -334,16 +334,8 @@ public void replaceRoots(Map, Operator> replacementMap) { public Set> getAllRootOperators() { Set> opSet = new LinkedHashSet>(); - Map> pa = getPathToAliases(); - if (pa != null) { - for (List ls : pa.values()) { - for (String a : ls) { - Operator op = getAliasToWork().get(a); - if (op != null ) { - opSet.add(op); - } - } - } + for (Operator op : getAliasToWork().values()) { + opSet.add(op); } return opSet; } diff --git a/ql/src/test/queries/clientpositive/tez_union.q b/ql/src/test/queries/clientpositive/tez_union.q index f51559f..96f58b2 100644 --- a/ql/src/test/queries/clientpositive/tez_union.q +++ b/ql/src/test/queries/clientpositive/tez_union.q @@ -92,3 +92,21 @@ right outer join src s on u.key = s.key; select * from ut order by ukey, skey limit 20; drop table ut; + +set hive.vectorized.execution.enabled=true; + +create table TABLE1(EMP_NAME STRING, EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','; + +create table table2 (EMP_NAME STRING) PARTITIONED BY (EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','; + +CREATE OR REPLACE VIEW TABLE3 as select EMP_NAME, EMP_ID from TABLE1; + +explain formatted select count(*) from TABLE3; + +drop table table2; + +create table table2 (EMP_NAME STRING) PARTITIONED BY (EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','; + +CREATE OR REPLACE VIEW TABLE3 as select EMP_NAME, EMP_ID from TABLE1 UNION ALL select EMP_NAME,EMP_ID from TABLE2; + +explain formatted select count(*) from TABLE3; diff --git a/ql/src/test/results/clientnegative/join_nonexistent_part.q.out b/ql/src/test/results/clientnegative/join_nonexistent_part.q.out index 7b31f23..391dd05 100644 --- a/ql/src/test/results/clientnegative/join_nonexistent_part.q.out +++ b/ql/src/test/results/clientnegative/join_nonexistent_part.q.out @@ -1,2 +1,2 @@ -Warning: Shuffle Join JOIN[8][tables = [$hdt$_0]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product Authorization failed:No privilege 'Select' found for inputs { database:default, table:srcpart, columnName:key}. Use SHOW GRANT to get more details. diff --git a/ql/src/test/results/clientpositive/annotate_stats_join.q.out b/ql/src/test/results/clientpositive/annotate_stats_join.q.out index 9f494b6..66e944b 100644 --- a/ql/src/test/results/clientpositive/annotate_stats_join.q.out +++ b/ql/src/test/results/clientpositive/annotate_stats_join.q.out @@ -456,22 +456,6 @@ STAGE PLANS: Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col2 (type: int) TableScan - alias: e - Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: deptid is not null (type: boolean) - Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: lastname (type: string), deptid (type: int), locid (type: int) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 48 Data size: 4752 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: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string), _col2 (type: int) - TableScan alias: d Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator @@ -487,6 +471,22 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string) + TableScan + alias: e + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: deptid is not null (type: boolean) + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: lastname (type: string), deptid (type: int), locid (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 48 Data size: 4752 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: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col2 (type: int) Reduce Operator Tree: Join Operator condition map: diff --git a/ql/src/test/results/clientpositive/auto_join32.q.out b/ql/src/test/results/clientpositive/auto_join32.q.out index d9fe656..bfc8be8 100644 --- a/ql/src/test/results/clientpositive/auto_join32.q.out +++ b/ql/src/test/results/clientpositive/auto_join32.q.out @@ -391,6 +391,36 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: v + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((p = 'bar') and name is not null) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: name (type: string), registration (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col3 + Select Operator + expressions: _col3 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: count(DISTINCT _col1) + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) Reduce Operator Tree: Group By Operator aggregations: count(DISTINCT KEY._col1:0._col0) diff --git a/ql/src/test/results/clientpositive/bucketmapjoin1.q.out b/ql/src/test/results/clientpositive/bucketmapjoin1.q.out index b8e4b41..72f2a07 100644 --- a/ql/src/test/results/clientpositive/bucketmapjoin1.q.out +++ b/ql/src/test/results/clientpositive/bucketmapjoin1.q.out @@ -125,6 +125,50 @@ STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Position of Big Table: 0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + BucketMapJoin: true + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col7 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2 + columns.types int:string:string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Local Work: Map Reduce Local Work @@ -249,6 +293,50 @@ STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key is not null and (ds = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Position of Big Table: 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + BucketMapJoin: true + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col7 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2 + columns.types int:string:string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Local Work: Map Reduce Local Work diff --git a/ql/src/test/results/clientpositive/correlationoptimizer3.q.out b/ql/src/test/results/clientpositive/correlationoptimizer3.q.out index 8b2d146..5389647 100644 --- a/ql/src/test/results/clientpositive/correlationoptimizer3.q.out +++ b/ql/src/test/results/clientpositive/correlationoptimizer3.q.out @@ -300,21 +300,6 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE TableScan - alias: y - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: key is not null (type: boolean) - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: key (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - TableScan alias: x Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE Filter Operator @@ -331,6 +316,21 @@ STAGE PLANS: Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) TableScan + alias: y + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + TableScan alias: x Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE Filter Operator @@ -1004,21 +1004,6 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE TableScan - alias: y - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: key is not null (type: boolean) - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: key (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - TableScan alias: x Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE Filter Operator @@ -1035,6 +1020,21 @@ STAGE PLANS: Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) TableScan + alias: y + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + TableScan alias: x Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE Filter Operator diff --git a/ql/src/test/results/clientpositive/correlationoptimizer6.q.out b/ql/src/test/results/clientpositive/correlationoptimizer6.q.out index 2e78dbe..be518dc 100644 --- a/ql/src/test/results/clientpositive/correlationoptimizer6.q.out +++ b/ql/src/test/results/clientpositive/correlationoptimizer6.q.out @@ -3032,22 +3032,6 @@ STAGE PLANS: Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) TableScan - alias: x - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: key is not null (type: boolean) - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) - TableScan alias: y Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE Filter Operator @@ -3069,6 +3053,22 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) + TableScan + alias: x + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) Reduce Operator Tree: Demux Operator Statistics: Num rows: 513 Data size: 5411 Basic stats: COMPLETE Column stats: NONE diff --git a/ql/src/test/results/clientpositive/groupby_sort_6.q.out b/ql/src/test/results/clientpositive/groupby_sort_6.q.out index 6ab652d..c5cb8b9 100644 --- a/ql/src/test/results/clientpositive/groupby_sort_6.q.out +++ b/ql/src/test/results/clientpositive/groupby_sort_6.q.out @@ -66,6 +66,33 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (ds = '1') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: false Needs Tagging: false Reduce Operator Tree: Group By Operator @@ -208,6 +235,33 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (ds = '1') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: false Needs Tagging: false Reduce Operator Tree: Group By Operator diff --git a/ql/src/test/results/clientpositive/input23.q.out b/ql/src/test/results/clientpositive/input23.q.out index 32b40cf..73038c3 100644 --- a/ql/src/test/results/clientpositive/input23.q.out +++ b/ql/src/test/results/clientpositive/input23.q.out @@ -1,4 +1,4 @@ -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain extended select * from srcpart a join srcpart b where a.ds = '2008-04-08' and a.hr = '11' and b.ds = '2008-04-08' and b.hr = '14' limit 5 PREHOOK: type: QUERY @@ -79,6 +79,24 @@ STAGE PLANS: tag: 0 value expressions: _col0 (type: string), _col1 (type: string) auto parallelism: false + TableScan + alias: a + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: ((ds = '2008-04-08') and (hr = '14')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 1 + value expressions: _col0 (type: string), _col1 (type: string) + auto parallelism: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -175,7 +193,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: select * from srcpart a join srcpart b where a.ds = '2008-04-08' and a.hr = '11' and b.ds = '2008-04-08' and b.hr = '14' limit 5 PREHOOK: type: QUERY PREHOOK: Input: default@srcpart diff --git a/ql/src/test/results/clientpositive/input26.q.out b/ql/src/test/results/clientpositive/input26.q.out index ef38d49..1b24aa6 100644 --- a/ql/src/test/results/clientpositive/input26.q.out +++ b/ql/src/test/results/clientpositive/input26.q.out @@ -87,6 +87,24 @@ STAGE PLANS: Stage: Stage-3 Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((ds = '2008-04-08') and (hr = '14')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Limit + Number of rows: 5 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: string), VALUE._col1 (type: string) diff --git a/ql/src/test/results/clientpositive/join_cond_pushdown_unqual2.q.out b/ql/src/test/results/clientpositive/join_cond_pushdown_unqual2.q.out index 1028f51..678ddb8 100644 --- a/ql/src/test/results/clientpositive/join_cond_pushdown_unqual2.q.out +++ b/ql/src/test/results/clientpositive/join_cond_pushdown_unqual2.q.out @@ -81,18 +81,6 @@ STAGE PLANS: Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) TableScan - alias: p4 - Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: p_name is not null (type: boolean) - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: p_name (type: string) - sort order: + - Map-reduce partition columns: p_name (type: string) - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE - value expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) - TableScan alias: p2 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE Filter Operator @@ -116,6 +104,18 @@ STAGE PLANS: Map-reduce partition columns: p3_name (type: string) Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE value expressions: p3_partkey (type: int), p3_mfgr (type: string), p3_brand (type: string), p3_type (type: string), p3_size (type: int), p3_container (type: string), p3_retailprice (type: double), p3_comment (type: string) + TableScan + alias: p4 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: p_name (type: string) + sort order: + + Map-reduce partition columns: p_name (type: string) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) Reduce Operator Tree: Join Operator condition map: diff --git a/ql/src/test/results/clientpositive/join_cond_pushdown_unqual4.q.out b/ql/src/test/results/clientpositive/join_cond_pushdown_unqual4.q.out index cb452bf..4668eb1 100644 --- a/ql/src/test/results/clientpositive/join_cond_pushdown_unqual4.q.out +++ b/ql/src/test/results/clientpositive/join_cond_pushdown_unqual4.q.out @@ -83,18 +83,6 @@ STAGE PLANS: Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) TableScan - alias: p4 - Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: p_name is not null (type: boolean) - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: p_name (type: string) - sort order: + - Map-reduce partition columns: p_name (type: string) - Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE - value expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) - TableScan alias: p2 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE Filter Operator @@ -118,6 +106,18 @@ STAGE PLANS: Map-reduce partition columns: p3_name (type: string) Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE value expressions: p3_partkey (type: int), p3_mfgr (type: string), p3_brand (type: string), p3_type (type: string), p3_size (type: int), p3_container (type: string), p3_retailprice (type: double), p3_comment (type: string) + TableScan + alias: p4 + Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: p_name is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: p_name (type: string) + sort order: + + Map-reduce partition columns: p_name (type: string) + Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE + value expressions: p_partkey (type: int), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) Reduce Operator Tree: Join Operator condition map: diff --git a/ql/src/test/results/clientpositive/join_view.q.out b/ql/src/test/results/clientpositive/join_view.q.out index ab28ff3..e703e0b 100644 --- a/ql/src/test/results/clientpositive/join_view.q.out +++ b/ql/src/test/results/clientpositive/join_view.q.out @@ -49,6 +49,31 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: invites + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '2011-09-01') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '2011-09-01' (type: string) + sort order: + + Map-reduce partition columns: '2011-09-01' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: bar (type: string) + TableScan + alias: invites2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '2011-09-01') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '2011-09-01' (type: string) + sort order: + + Map-reduce partition columns: '2011-09-01' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: foo (type: int) Reduce Operator Tree: Join Operator condition map: diff --git a/ql/src/test/results/clientpositive/metadataonly1.q.out b/ql/src/test/results/clientpositive/metadataonly1.q.out index b4d2617..e55efd5 100644 --- a/ql/src/test/results/clientpositive/metadataonly1.q.out +++ b/ql/src/test/results/clientpositive/metadataonly1.q.out @@ -36,6 +36,26 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: test1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col0 (type: string) + auto parallelism: false Needs Tagging: false Reduce Operator Tree: Group By Operator diff --git a/ql/src/test/results/clientpositive/nullgroup5.q.out b/ql/src/test/results/clientpositive/nullgroup5.q.out index ddf8729..8a94d62 100644 --- a/ql/src/test/results/clientpositive/nullgroup5.q.out +++ b/ql/src/test/results/clientpositive/nullgroup5.q.out @@ -57,6 +57,25 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan + alias: x + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '2009-04-05') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan alias: y Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE Select Operator diff --git a/ql/src/test/results/clientpositive/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/optimize_nullscan.q.out index 609e415..1f4becf 100644 --- a/ql/src/test/results/clientpositive/optimize_nullscan.q.out +++ b/ql/src/test/results/clientpositive/optimize_nullscan.q.out @@ -176,6 +176,29 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: false Needs Tagging: false Reduce Operator Tree: Group By Operator @@ -535,15 +558,6 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan - GatherStats: false - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE - tag: 1 - auto parallelism: false - TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE GatherStats: false @@ -562,6 +576,15 @@ STAGE PLANS: Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE tag: 0 auto parallelism: false + TableScan + GatherStats: false + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 1 + auto parallelism: false Path -> Alias: -mr-10003default.src{} [a:src] #### A masked pattern was here #### @@ -1511,14 +1534,6 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan - GatherStats: false - Reduce Output Operator - sort order: - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE - tag: 1 - value expressions: _col0 (type: string) - auto parallelism: false - TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE GatherStats: false @@ -1536,6 +1551,14 @@ STAGE PLANS: tag: 0 value expressions: _col0 (type: string) auto parallelism: false + TableScan + GatherStats: false + Reduce Output Operator + sort order: + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 1 + value expressions: _col0 (type: string) + auto parallelism: false Path -> Alias: -mr-10003default.src{} [a:src] #### A masked pattern was here #### @@ -1752,6 +1775,21 @@ STAGE PLANS: Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE tag: 0 auto parallelism: false + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 1 + auto parallelism: false Path -> Alias: -mr-10004default.src{} [null-subquery2:a-subquery2:src] Path -> Partition: @@ -1832,31 +1870,6 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan - GatherStats: false - Union - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE -#### A masked pattern was here #### - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - columns _col0 - columns.types string - escape.delim \ - hive.serialization.extend.additional.nesting.levels true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE GatherStats: false @@ -1891,6 +1904,31 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false + TableScan + GatherStats: false + Union + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Path -> Alias: -mr-10003default.src{} [null-subquery1:a-subquery1:src] #### A masked pattern was here #### diff --git a/ql/src/test/results/clientpositive/partition_boolexpr.q.out b/ql/src/test/results/clientpositive/partition_boolexpr.q.out index 90412eb..cfd03e2 100644 --- a/ql/src/test/results/clientpositive/partition_boolexpr.q.out +++ b/ql/src/test/results/clientpositive/partition_boolexpr.q.out @@ -84,6 +84,22 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) @@ -253,6 +269,22 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) diff --git a/ql/src/test/results/clientpositive/ppd_union_view.q.out b/ql/src/test/results/clientpositive/ppd_union_view.q.out index b7bfeaf..a13ef7a 100644 --- a/ql/src/test/results/clientpositive/ppd_union_view.q.out +++ b/ql/src/test/results/clientpositive/ppd_union_view.q.out @@ -332,6 +332,45 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan + alias: t1_new + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (ds = '2011-10-13') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 1 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), '2011-10-13' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 15 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 1 Data size: 15 Basic stats: COMPLETE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2 + columns.types string:string:string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + TableScan GatherStats: false Union Statistics: Num rows: 1 Data size: 15 Basic stats: COMPLETE Column stats: NONE @@ -465,6 +504,47 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-3 Map Reduce + Map Operator Tree: + TableScan + alias: t1_old + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: ((ds = '2011-10-15') and keymap is not null) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: keymap (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 0 + value expressions: _col1 (type: string) + auto parallelism: false + TableScan + alias: t1_mapping + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (('2011-10-15' = ds) and keymap is not null) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string), keymap (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 1 + value expressions: _col0 (type: string) + auto parallelism: false Needs Tagging: true Reduce Operator Tree: Join Operator diff --git a/ql/src/test/results/clientpositive/sample6.q.out b/ql/src/test/results/clientpositive/sample6.q.out index e1914fb..9891cbb 100644 --- a/ql/src/test/results/clientpositive/sample6.q.out +++ b/ql/src/test/results/clientpositive/sample6.q.out @@ -3195,6 +3195,26 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: s + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: true + predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) + sampleDesc: BUCKET 1 OUT OF 2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: -1 + auto parallelism: false Needs Tagging: false Reduce Operator Tree: Select Operator diff --git a/ql/src/test/results/clientpositive/smb_mapjoin9.q.out b/ql/src/test/results/clientpositive/smb_mapjoin9.q.out index 0c3771a..9530be1 100644 --- a/ql/src/test/results/clientpositive/smb_mapjoin9.q.out +++ b/ql/src/test/results/clientpositive/smb_mapjoin9.q.out @@ -112,6 +112,46 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key is not null and (ds = '2010-10-15')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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) + outputColumnNames: _col0, _col6, _col7 + Position of Big Table: 0 + Select Operator + expressions: _col6 (type: int), _col7 (type: string), '2010-10-15' (type: string), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2,_col3 + columns.types int:string:string:int + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-0 Fetch Operator @@ -231,6 +271,46 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Map Reduce + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key is not null and (ds = '2010-10-15')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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) + outputColumnNames: _col0, _col6, _col7 + Position of Big Table: 1 + Select Operator + expressions: _col6 (type: int), _col7 (type: string), '2010-10-15' (type: string), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2,_col3 + columns.types int:string:string:int + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/spark/auto_join32.q.out b/ql/src/test/results/clientpositive/spark/auto_join32.q.out index e26cefe..c537b95 100644 --- a/ql/src/test/results/clientpositive/spark/auto_join32.q.out +++ b/ql/src/test/results/clientpositive/spark/auto_join32.q.out @@ -426,6 +426,32 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: s + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (name is not null and (p = 'bar')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 name (type: string) + 1 name (type: string) + outputColumnNames: _col0, _col9 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(DISTINCT _col9) + keys: _col0 (type: string), _col9 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE Reducer 2 Reduce Operator Tree: Group By Operator diff --git a/ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out b/ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out index f6c9149..44f4d0c 100644 --- a/ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out +++ b/ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out @@ -101,6 +101,20 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 2 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key is not null and (ds = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + Position of Big Table: 0 Local Work: Map Reduce Local Work Bucket Mapjoin Context: @@ -112,6 +126,52 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + input vertices: + 1 Map 2 + Position of Big Table: 0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + BucketMapJoin: true + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col7 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2 + columns.types int:string:string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Local Work: Map Reduce Local Work Bucket Mapjoin Context: @@ -215,6 +275,20 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + Position of Big Table: 1 Local Work: Map Reduce Local Work Bucket Mapjoin Context: @@ -226,6 +300,52 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 2 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key is not null and (ds = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + input vertices: + 0 Map 1 + Position of Big Table: 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + BucketMapJoin: true + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col7 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2 + columns.types int:string:string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Local Work: Map Reduce Local Work Bucket Mapjoin Context: diff --git a/ql/src/test/results/clientpositive/spark/join_view.q.out b/ql/src/test/results/clientpositive/spark/join_view.q.out index eee392d..f6e0542 100644 --- a/ql/src/test/results/clientpositive/spark/join_view.q.out +++ b/ql/src/test/results/clientpositive/spark/join_view.q.out @@ -54,7 +54,33 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: invites + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '2011-09-01') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '2011-09-01' (type: string) + sort order: + + Map-reduce partition columns: '2011-09-01' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: bar (type: string) Map 3 + Map Operator Tree: + TableScan + alias: invites2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '2011-09-01') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '2011-09-01' (type: string) + sort order: + + Map-reduce partition columns: '2011-09-01' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: foo (type: int) Reducer 2 Reduce Operator Tree: Join Operator diff --git a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out index 881f41a..843570e 100644 --- a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out +++ b/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out @@ -107,6 +107,29 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: false Reducer 2 Needs Tagging: false Reduce Operator Tree: @@ -1654,6 +1677,22 @@ STAGE PLANS: Truncated Path -> Alias: -mr-10003default.src{} [src] Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 1 + auto parallelism: false Reducer 3 Needs Tagging: true Reduce Operator Tree: diff --git a/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out b/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out index acf7211..8d3f56b 100644 --- a/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out +++ b/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out @@ -393,6 +393,40 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: complex_tbl_2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (ds = '2010-03-29') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: aet (type: string), aes (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Transform Operator + command: cat + output info: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2,_col3,_col4,_col5,_col6 + columns.types string,string,int,string,bigint,string,string + field.delim 9 + serialization.format 9 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: -1 + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) + auto parallelism: false Reducer 2 Needs Tagging: false Reduce Operator Tree: diff --git a/ql/src/test/results/clientpositive/spark/sample6.q.out b/ql/src/test/results/clientpositive/spark/sample6.q.out index a44d21f..4117732 100644 --- a/ql/src/test/results/clientpositive/spark/sample6.q.out +++ b/ql/src/test/results/clientpositive/spark/sample6.q.out @@ -3081,6 +3081,26 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: s + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: true + predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) + sampleDesc: BUCKET 1 OUT OF 2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: -1 + auto parallelism: false Reducer 2 Needs Tagging: false Reduce Operator Tree: diff --git a/ql/src/test/results/clientpositive/spark/union_view.q.out b/ql/src/test/results/clientpositive/spark/union_view.q.out index 593ce40..2ca9e13 100644 --- a/ql/src/test/results/clientpositive/spark/union_view.q.out +++ b/ql/src/test/results/clientpositive/spark/union_view.q.out @@ -290,7 +290,43 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Map 2 + Map Operator Tree: + TableScan + alias: src_union_2 + filterExpr: ((key = 86) and (ds = '1')) (type: boolean) + Filter Operator + predicate: ((key = 86) and (ds = '1')) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Select Operator + expressions: 86 (type: int), _col1 (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + 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 Map 3 + Map Operator Tree: + TableScan + alias: src_union_3 + filterExpr: ((key = 86) and (ds = '1')) (type: boolean) + Filter Operator + predicate: ((key = 86) and (ds = '1')) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Select Operator + expressions: 86 (type: int), _col1 (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Stage: Stage-0 Fetch Operator @@ -308,6 +344,24 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: src_union_1 + filterExpr: ((key = 86) and (ds = '2')) (type: boolean) + Filter Operator + predicate: ((key = 86) and (ds = '2')) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Select Operator + expressions: 86 (type: int), _col1 (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + 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 Map 2 Map Operator Tree: TableScan @@ -328,6 +382,24 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Map 3 + Map Operator Tree: + TableScan + alias: src_union_3 + filterExpr: ((key = 86) and (ds = '2')) (type: boolean) + Filter Operator + predicate: ((key = 86) and (ds = '2')) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Select Operator + expressions: 86 (type: int), _col1 (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Stage: Stage-0 Fetch Operator @@ -345,7 +417,43 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: src_union_1 + filterExpr: ((key = 86) and (ds = '3')) (type: boolean) + Filter Operator + predicate: ((key = 86) and (ds = '3')) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Select Operator + expressions: 86 (type: int), _col1 (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + 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 Map 2 + Map Operator Tree: + TableScan + alias: src_union_2 + filterExpr: ((key = 86) and (ds = '3')) (type: boolean) + Filter Operator + predicate: ((key = 86) and (ds = '3')) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Select Operator + expressions: 86 (type: int), _col1 (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + 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 Map 3 Map Operator Tree: TableScan @@ -490,7 +598,37 @@ STAGE PLANS: sort order: value expressions: _col0 (type: bigint) Map 3 + Map Operator Tree: + TableScan + alias: src_union_2 + filterExpr: (ds = '1') (type: boolean) + Filter Operator + predicate: (ds = '1') (type: boolean) + Select Operator + Select Operator + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) Map 4 + Map Operator Tree: + TableScan + alias: src_union_3 + filterExpr: (ds = '1') (type: boolean) + Filter Operator + predicate: (ds = '1') (type: boolean) + Select Operator + Select Operator + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) Reducer 2 Reduce Operator Tree: Group By Operator @@ -524,6 +662,21 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: src_union_1 + filterExpr: (ds = '2') (type: boolean) + Filter Operator + predicate: (ds = '2') (type: boolean) + Select Operator + Select Operator + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) Map 3 Map Operator Tree: TableScan @@ -539,6 +692,21 @@ STAGE PLANS: sort order: value expressions: _col0 (type: bigint) Map 4 + Map Operator Tree: + TableScan + alias: src_union_3 + filterExpr: (ds = '2') (type: boolean) + Filter Operator + predicate: (ds = '2') (type: boolean) + Select Operator + Select Operator + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) Reducer 2 Reduce Operator Tree: Group By Operator @@ -572,7 +740,37 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: src_union_1 + filterExpr: (ds = '3') (type: boolean) + Filter Operator + predicate: (ds = '3') (type: boolean) + Select Operator + Select Operator + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) Map 3 + Map Operator Tree: + TableScan + alias: src_union_2 + filterExpr: (ds = '3') (type: boolean) + Filter Operator + predicate: (ds = '3') (type: boolean) + Select Operator + Select Operator + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) Map 4 Map Operator Tree: TableScan @@ -621,7 +819,43 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: src_union_1 + filterExpr: ((key = 86) and (ds = '4')) (type: boolean) + Filter Operator + predicate: ((key = 86) and (ds = '4')) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Select Operator + expressions: 86 (type: int), _col1 (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + 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 Map 2 + Map Operator Tree: + TableScan + alias: src_union_2 + filterExpr: ((key = 86) and (ds = '4')) (type: boolean) + Filter Operator + predicate: ((key = 86) and (ds = '4')) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Select Operator + expressions: 86 (type: int), _col1 (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + 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 Map 3 Map Operator Tree: TableScan @@ -661,7 +895,37 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: src_union_1 + filterExpr: (ds = '4') (type: boolean) + Filter Operator + predicate: (ds = '4') (type: boolean) + Select Operator + Select Operator + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) Map 3 + Map Operator Tree: + TableScan + alias: src_union_2 + filterExpr: (ds = '4') (type: boolean) + Filter Operator + predicate: (ds = '4') (type: boolean) + Select Operator + Select Operator + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) Map 4 Map Operator Tree: TableScan diff --git a/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out index 30847f9..9a04fa2 100644 --- a/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out +++ b/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out @@ -2588,6 +2588,19 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE Map 5 Map Operator Tree: TableScan @@ -4595,6 +4608,28 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Map-reduce partition columns: '13' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE Map 2 Map Operator Tree: TableScan diff --git a/ql/src/test/results/clientpositive/tez/metadataonly1.q.out b/ql/src/test/results/clientpositive/tez/metadataonly1.q.out index a595c4c..2850314 100644 --- a/ql/src/test/results/clientpositive/tez/metadataonly1.q.out +++ b/ql/src/test/results/clientpositive/tez/metadataonly1.q.out @@ -41,6 +41,26 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: test1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col0 (type: string) + auto parallelism: false Reducer 2 Needs Tagging: false Reduce Operator Tree: diff --git a/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out index 104654a..cca489e 100644 --- a/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out +++ b/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out @@ -104,6 +104,29 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: true Reducer 2 Needs Tagging: false Reduce Operator Tree: @@ -1656,6 +1679,22 @@ STAGE PLANS: Truncated Path -> Alias: -mr-10002default.src{} [src] Map 5 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 1 + auto parallelism: true Reducer 4 Needs Tagging: false Reduce Operator Tree: diff --git a/ql/src/test/results/clientpositive/tez/tez_union.q.out b/ql/src/test/results/clientpositive/tez/tez_union.q.out index 2472c5e..6f6e8ca 100644 --- a/ql/src/test/results/clientpositive/tez/tez_union.q.out +++ b/ql/src/test/results/clientpositive/tez/tez_union.q.out @@ -1253,3 +1253,68 @@ POSTHOOK: query: drop table ut POSTHOOK: type: DROPTABLE POSTHOOK: Input: default@ut POSTHOOK: Output: default@ut +PREHOOK: query: create table TABLE1(EMP_NAME STRING, EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@TABLE1 +POSTHOOK: query: create table TABLE1(EMP_NAME STRING, EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@TABLE1 +PREHOOK: query: create table table2 (EMP_NAME STRING) PARTITIONED BY (EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table2 +POSTHOOK: query: create table table2 (EMP_NAME STRING) PARTITIONED BY (EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table2 +PREHOOK: query: CREATE OR REPLACE VIEW TABLE3 as select EMP_NAME, EMP_ID from TABLE1 +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@table1 +PREHOOK: Output: database:default +PREHOOK: Output: default@TABLE3 +POSTHOOK: query: CREATE OR REPLACE VIEW TABLE3 as select EMP_NAME, EMP_ID from TABLE1 +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@table1 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@TABLE3 +PREHOOK: query: explain formatted select count(*) from TABLE3 +PREHOOK: type: QUERY +POSTHOOK: query: explain formatted select count(*) from TABLE3 +POSTHOOK: type: QUERY +#### A masked pattern was here #### +PREHOOK: query: drop table table2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@table2 +PREHOOK: Output: default@table2 +POSTHOOK: query: drop table table2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@table2 +POSTHOOK: Output: default@table2 +PREHOOK: query: create table table2 (EMP_NAME STRING) PARTITIONED BY (EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table2 +POSTHOOK: query: create table table2 (EMP_NAME STRING) PARTITIONED BY (EMP_ID INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table2 +PREHOOK: query: CREATE OR REPLACE VIEW TABLE3 as select EMP_NAME, EMP_ID from TABLE1 UNION ALL select EMP_NAME,EMP_ID from TABLE2 +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@table1 +PREHOOK: Input: default@table2 +PREHOOK: Output: database:default +PREHOOK: Output: default@TABLE3 +POSTHOOK: query: CREATE OR REPLACE VIEW TABLE3 as select EMP_NAME, EMP_ID from TABLE1 UNION ALL select EMP_NAME,EMP_ID from TABLE2 +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@table1 +POSTHOOK: Input: default@table2 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@TABLE3 +POSTHOOK: Output: default@table3 +PREHOOK: query: explain formatted select count(*) from TABLE3 +PREHOOK: type: QUERY +POSTHOOK: query: explain formatted select count(*) from TABLE3 +POSTHOOK: type: QUERY +#### A masked pattern was here #### diff --git a/ql/src/test/results/clientpositive/tez/tez_union_group_by.q.out b/ql/src/test/results/clientpositive/tez/tez_union_group_by.q.out index 844ba23..654b34b 100644 --- a/ql/src/test/results/clientpositive/tez/tez_union_group_by.q.out +++ b/ql/src/test/results/clientpositive/tez/tez_union_group_by.q.out @@ -156,6 +156,24 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: x + Filter Operator + predicate: ((date < '2014-09-02') and (u <> 0)) (type: boolean) + Select Operator + expressions: u (type: bigint), date (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: min(_col1) + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + value expressions: _col1 (type: string) Map 10 Map Operator Tree: TableScan @@ -170,8 +188,57 @@ STAGE PLANS: Map-reduce partition columns: t (type: string), st (type: string) Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE Map 5 + Map Operator Tree: + TableScan + alias: y + Filter Operator + predicate: ((date < '2014-09-02') and (u <> 0)) (type: boolean) + Select Operator + expressions: u (type: bigint), date (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: min(_col1) + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + value expressions: _col1 (type: string) Map 6 + Map Operator Tree: + TableScan + alias: z + Filter Operator + predicate: ((date < '2014-09-02') and (u <> 0)) (type: boolean) + Select Operator + expressions: u (type: bigint), date (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: min(_col1) + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + value expressions: _col1 (type: string) Map 7 + Map Operator Tree: + TableScan + alias: x + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (((t is not null and (date >= '2014-03-04')) and (date < '2014-09-03')) and (u <> 0)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: t (type: string), st (type: string) + sort order: ++ + Map-reduce partition columns: t (type: string), st (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: u (type: bigint) Reducer 3 Reduce Operator Tree: Group By Operator diff --git a/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out index 81e2855..a8b25db 100644 --- a/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out +++ b/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out @@ -2620,6 +2620,19 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE Map 5 Map Operator Tree: TableScan @@ -4652,6 +4665,28 @@ STAGE PLANS: #### A masked pattern was here #### Vertices: Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Map-reduce partition columns: '13' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE Map 2 Map Operator Tree: TableScan diff --git a/ql/src/test/results/clientpositive/union30.q.out b/ql/src/test/results/clientpositive/union30.q.out index f65dfc4..4529074 100644 --- a/ql/src/test/results/clientpositive/union30.q.out +++ b/ql/src/test/results/clientpositive/union30.q.out @@ -117,27 +117,6 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.union_subq_union 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 - Union - Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.union_subq_union - TableScan Union Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE Select Operator @@ -167,6 +146,27 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.union_subq_union + 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 + Union + Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.union_subq_union Stage: Stage-8 Conditional Operator diff --git a/ql/src/test/results/clientpositive/union_lateralview.q.out b/ql/src/test/results/clientpositive/union_lateralview.q.out index 1975ac3..734c1f4 100644 --- a/ql/src/test/results/clientpositive/union_lateralview.q.out +++ b/ql/src/test/results/clientpositive/union_lateralview.q.out @@ -102,14 +102,6 @@ STAGE PLANS: Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col2 (type: string) TableScan - alias: b - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - TableScan alias: srcpart Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator @@ -157,6 +149,14 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col2 (type: string) + TableScan + alias: b + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Join Operator condition map: diff --git a/ql/src/test/results/clientpositive/union_view.q.out b/ql/src/test/results/clientpositive/union_view.q.out index 6c99491..81ec9e7 100644 --- a/ql/src/test/results/clientpositive/union_view.q.out +++ b/ql/src/test/results/clientpositive/union_view.q.out @@ -487,6 +487,54 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan + alias: src_union_2 + filterExpr: ((key = 86) and (ds = '1')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((key = 86) and (ds = '1')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 86 (type: int), _col1 (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan + alias: src_union_3 + filterExpr: ((key = 86) and (ds = '1')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((key = 86) and (ds = '1')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 86 (type: int), _col1 (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Stage: Stage-0 Fetch Operator @@ -529,6 +577,30 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan + alias: src_union_1 + filterExpr: ((key = 86) and (ds = '2')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((key = 86) and (ds = '2')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 86 (type: int), _col1 (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + 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.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan alias: src_union_2 filterExpr: ((key = 86) and (ds = '2')) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE @@ -552,6 +624,30 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan + alias: src_union_3 + filterExpr: ((key = 86) and (ds = '2')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((key = 86) and (ds = '2')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 86 (type: int), _col1 (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + 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.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Stage: Stage-0 Fetch Operator @@ -594,6 +690,54 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan + alias: src_union_1 + filterExpr: ((key = 86) and (ds = '3')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((key = 86) and (ds = '3')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 86 (type: int), _col1 (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 + 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.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan + alias: src_union_2 + filterExpr: ((key = 86) and (ds = '3')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((key = 86) and (ds = '3')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 86 (type: int), _col1 (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 + 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.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan alias: src_union_3 filterExpr: ((key = 86) and (ds = '3')) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE @@ -856,6 +1000,50 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) + TableScan + alias: src_union_2 + filterExpr: (ds = '1') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '1') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 5312 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) + TableScan + alias: src_union_3 + filterExpr: (ds = '1') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '1') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 5312 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) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) @@ -909,6 +1097,28 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan + alias: src_union_1 + filterExpr: (ds = '2') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '2') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1000 Data size: 10624 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) + TableScan alias: src_union_2 filterExpr: (ds = '2') (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE @@ -927,6 +1137,28 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) + TableScan + alias: src_union_3 + filterExpr: (ds = '2') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '2') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1000 Data size: 10624 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) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) @@ -980,6 +1212,50 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan + alias: src_union_1 + filterExpr: (ds = '3') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '3') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1000 Data size: 10624 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) + TableScan + alias: src_union_2 + filterExpr: (ds = '3') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '3') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1000 Data size: 10624 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) + TableScan alias: src_union_3 filterExpr: (ds = '3') (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE @@ -1056,6 +1332,54 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan + alias: src_union_1 + filterExpr: ((key = 86) and (ds = '4')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((key = 86) and (ds = '4')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 86 (type: int), _col1 (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan + alias: src_union_2 + filterExpr: ((key = 86) and (ds = '4')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((key = 86) and (ds = '4')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 86 (type: int), _col1 (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TableScan alias: src_union_3 filterExpr: ((key = 86) and (ds = '4')) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE @@ -1120,6 +1444,50 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan + alias: src_union_1 + filterExpr: (ds = '4') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '4') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 5312 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) + TableScan + alias: src_union_2 + filterExpr: (ds = '4') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (ds = '4') (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Union + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 5312 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) + TableScan alias: src_union_3 filterExpr: (ds = '4') (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE