Index: conf/hive-default.xml.template =================================================================== --- conf/hive-default.xml.template (revision 1425241) +++ conf/hive-default.xml.template (working copy) @@ -526,6 +526,22 @@ + hive.new.job.grouping.set.cardinality + 30 + + Whether a new map-reduce job should be launched for grouping sets/rollups/cubes. + For a query like: select a, b, c, count(1) from T group by a, b, c with rollup; + 4 rows are created per row: (a, b, c), (a, b, null), (a, null, null), (null, null, null). + This can lead to explosion across map-reduce boundary if the cardinality of T is very high, + and map-side aggregation does not do a very good job. + + This parameter decides if hive should add an additional map-reduce job. If the grouping set + cardinality (4 in the example above), is more than this value, a new MR job is added under the + assumption that the orginal group by will reduce the data size. + + + + hive.join.emit.interval 1000 How many rows in the right-most join operand Hive should buffer before emitting the join result. Index: data/files/grouping_sets1.txt =================================================================== --- data/files/grouping_sets1.txt (revision 0) +++ data/files/grouping_sets1.txt (working copy) @@ -0,0 +1,6 @@ +8 1 1 +5 1 2 +1 1 3 +2 2 4 +2 3 5 +3 2 8 Index: data/files/grouping_sets2.txt =================================================================== --- data/files/grouping_sets2.txt (revision 0) +++ data/files/grouping_sets2.txt (working copy) @@ -0,0 +1,6 @@ +8 1 1 +1 2 2 +1 1 3 +2 2 4 +2 3 5 +2 2 8 Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java =================================================================== --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (revision 1425241) +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (working copy) @@ -406,6 +406,7 @@ HIVEMAPAGGRHASHMINREDUCTION("hive.map.aggr.hash.min.reduction", (float) 0.5), HIVEMULTIGROUPBYSINGLEREDUCER("hive.multigroupby.singlereducer", true), HIVE_MAP_GROUPBY_SORT("hive.map.groupby.sorted", false), + HIVE_NEW_JOB_GROUPING_SET_CARDINALITY("hive.new.job.grouping.set.cardinality", 30), // for hive udtf operator HIVEUDTFAUTOPROGRESS("hive.udtf.auto.progress", false), Index: ql/src/test/results/clientpositive/groupby_grouping_sets4.q.out =================================================================== --- ql/src/test/results/clientpositive/groupby_grouping_sets4.q.out (revision 0) +++ ql/src/test/results/clientpositive/groupby_grouping_sets4.q.out (working copy) @@ -0,0 +1,673 @@ +PREHOOK: query: -- Set merging to false above to make the explain more readable + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- Set merging to false above to make the explain more readable + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1 +PREHOOK: type: LOAD +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t1 +PREHOOK: query: -- This tests that cubes and rollups work fine inside sub-queries. +EXPLAIN +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a +PREHOOK: type: QUERY +POSTHOOK: query: -- This tests that cubes and rollups work fine inside sub-queries. +EXPLAIN +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_WHERE (< (TOK_TABLE_OR_COL a) 3)) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) subq1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_WHERE (< (TOK_TABLE_OR_COL a) 3)) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) subq2) (= (. (TOK_TABLE_OR_COL subq1) a) (. (TOK_TABLE_OR_COL subq2) a)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + subq1:t1 + TableScan + alias: t1 + Filter Operator + predicate: + expr: (a < 3.0) + type: boolean + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + outputColumnNames: a, b + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + expr: '0' + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + subq2:t1 + TableScan + alias: t1 + Filter Operator + predicate: + expr: (a < 3.0) + type: boolean + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + outputColumnNames: a, b + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + expr: '0' + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +1 NULL 1 1 NULL 1 +1 NULL 1 1 1 1 +1 1 1 1 NULL 1 +1 1 1 1 1 1 +2 NULL 2 2 NULL 2 +2 NULL 2 2 2 1 +2 NULL 2 2 3 1 +2 2 1 2 NULL 2 +2 2 1 2 2 1 +2 2 1 2 3 1 +2 3 1 2 NULL 2 +2 3 1 2 2 1 +2 3 1 2 3 1 +PREHOOK: query: -- Since 4 grouping sets would be generated for each sub-query, an additional MR job should be created +-- for each of them +EXPLAIN +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a +PREHOOK: type: QUERY +POSTHOOK: query: -- Since 4 grouping sets would be generated for each sub-query, an additional MR job should be created +-- for each of them +EXPLAIN +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_WHERE (< (TOK_TABLE_OR_COL a) 3)) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) subq1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_WHERE (< (TOK_TABLE_OR_COL a) 3)) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) subq2) (= (. (TOK_TABLE_OR_COL subq1) a) (. (TOK_TABLE_OR_COL subq2) a)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2, Stage-5 + Stage-4 is a root stage + Stage-5 depends on stages: Stage-4 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + subq1:t1 + TableScan + alias: t1 + Filter Operator + predicate: + expr: (a < 3.0) + type: boolean + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + outputColumnNames: a, b + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: '0' + type: string + mode: partials + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: final + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: + subq2:t1 + TableScan + alias: t1 + Filter Operator + predicate: + expr: (a < 3.0) + type: boolean + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + outputColumnNames: a, b + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: '0' + type: string + mode: partials + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: final + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +1 NULL 1 1 NULL 1 +1 NULL 1 1 1 1 +1 1 1 1 NULL 1 +1 1 1 1 1 1 +2 NULL 2 2 NULL 2 +2 NULL 2 2 2 1 +2 NULL 2 2 3 1 +2 2 1 2 NULL 2 +2 2 1 2 2 1 +2 2 1 2 3 1 +2 3 1 2 NULL 2 +2 3 1 2 2 1 +2 3 1 2 3 1 Index: ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out =================================================================== --- ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out (revision 0) +++ ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out (working copy) @@ -0,0 +1,332 @@ +PREHOOK: query: -- In this test, 2 files are loaded into table T1. The data contains rows with the same value of a and b, +-- with different number of rows for a and b in each file. Since bucketizedHiveInputFormat is used, +-- this tests that the aggregate function stores the partial aggregate state correctly even if an +-- additional MR job is created for processing the grouping sets. +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- In this test, 2 files are loaded into table T1. The data contains rows with the same value of a and b, +-- with different number of rows for a and b in each file. Since bucketizedHiveInputFormat is used, +-- this tests that the aggregate function stores the partial aggregate state correctly even if an +-- additional MR job is created for processing the grouping sets. +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets2.txt' INTO TABLE T1 +PREHOOK: type: LOAD +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets2.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t1 +PREHOOK: query: -- The query below will execute in a single MR job, since 4 rows are generated per input row +-- (cube of a,b will lead to (a,b), (a, null), (null, b) and (null, null) and +-- hive.new.job.grouping.set.cardinality is more than 4. +EXPLAIN +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube +PREHOOK: type: QUERY +POSTHOOK: query: -- The query below will execute in a single MR job, since 4 rows are generated per input row +-- (cube of a,b will lead to (a,b), (a, null), (null, b) and (null, null) and +-- hive.new.job.grouping.set.cardinality is more than 4. +EXPLAIN +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTION avg (TOK_TABLE_OR_COL c))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + t1 + TableScan + alias: t1 + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + expr: c + type: string + outputColumnNames: a, b, c + Group By Operator + aggregations: + expr: avg(c) + expr: count() + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + expr: '0' + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: struct + expr: _col4 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: avg(VALUE._col0) + expr: count(VALUE._col1) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: double + expr: _col4 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT a, b, avg(c), count(*) from T1 group by a, b with cube +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT a, b, avg(c), count(*) from T1 group by a, b with cube +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +NULL NULL 3.8333333333333335 12 +NULL 1 2.0 5 +NULL 2 5.2 5 +NULL 3 5.0 2 +1 NULL 2.6666666666666665 3 +1 1 3.0 2 +1 2 2.0 1 +2 NULL 5.2 5 +2 2 5.333333333333333 3 +2 3 5.0 2 +3 NULL 8.0 1 +3 2 8.0 1 +5 NULL 2.0 1 +5 1 2.0 1 +8 NULL 1.0 2 +8 1 1.0 2 +PREHOOK: query: -- The query below will execute in 2 MR jobs, since hive.new.job.grouping.set.cardinality is set to 2. +-- The partial aggregation state should be maintained correctly across MR jobs. +EXPLAIN +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube +PREHOOK: type: QUERY +POSTHOOK: query: -- The query below will execute in 2 MR jobs, since hive.new.job.grouping.set.cardinality is set to 2. +-- The partial aggregation state should be maintained correctly across MR jobs. +EXPLAIN +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTION avg (TOK_TABLE_OR_COL c))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + t1 + TableScan + alias: t1 + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + expr: c + type: string + outputColumnNames: a, b, c + Group By Operator + aggregations: + expr: avg(c) + expr: count() + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: struct + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: avg(VALUE._col0) + expr: count(VALUE._col1) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: '0' + type: string + mode: partials + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: struct + expr: _col4 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: avg(VALUE._col0) + expr: count(VALUE._col1) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: final + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: double + expr: _col4 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT a, b, avg(c), count(*) from T1 group by a, b with cube +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT a, b, avg(c), count(*) from T1 group by a, b with cube +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +NULL NULL 3.8333333333333335 12 +NULL 1 2.0 5 +NULL 2 5.2 5 +NULL 3 5.0 2 +1 NULL 2.6666666666666665 3 +1 1 3.0 2 +1 2 2.0 1 +2 NULL 5.2 5 +2 2 5.333333333333333 3 +2 3 5.0 2 +3 NULL 8.0 1 +3 2 8.0 1 +5 NULL 2.0 1 +5 1 2.0 1 +8 NULL 1.0 2 +8 1 1.0 2 Index: ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out =================================================================== --- ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out (revision 0) +++ ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out (working copy) @@ -0,0 +1,520 @@ +PREHOOK: query: CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1 +PREHOOK: type: LOAD +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t1 +PREHOOK: query: -- Since 4 grouping sets would be generated for the query below, an additional MR job should be created +EXPLAIN +SELECT a, b, count(*) from T1 group by a, b with cube +PREHOOK: type: QUERY +POSTHOOK: query: -- Since 4 grouping sets would be generated for the query below, an additional MR job should be created +EXPLAIN +SELECT a, b, count(*) from T1 group by a, b with cube +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + t1 + TableScan + alias: t1 + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + outputColumnNames: a, b + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: '0' + type: string + mode: partials + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: final + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT a, b, count(*) from T1 group by a, b with cube +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT a, b, count(*) from T1 group by a, b with cube +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +NULL NULL 6 +NULL 1 2 +NULL 2 3 +NULL 3 1 +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 +PREHOOK: query: EXPLAIN +SELECT a, b, sum(c) from T1 group by a, b with cube +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT a, b, sum(c) from T1 group by a, b with cube +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL c)))) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + t1 + TableScan + alias: t1 + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + expr: c + type: string + outputColumnNames: a, b, c + Group By Operator + aggregations: + expr: sum(c) + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: double + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: '0' + type: string + mode: partials + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: double + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: final + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: double + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT a, b, sum(c) from T1 group by a, b with cube +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT a, b, sum(c) from T1 group by a, b with cube +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +NULL NULL 23.0 +NULL 1 4.0 +NULL 2 14.0 +NULL 3 5.0 +1 NULL 3.0 +1 1 3.0 +2 NULL 9.0 +2 2 4.0 +2 3 5.0 +3 NULL 8.0 +3 2 8.0 +5 NULL 2.0 +5 2 2.0 +8 NULL 1.0 +8 1 1.0 +PREHOOK: query: CREATE TABLE T2(a STRING, b STRING, c int, d int) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T2(a STRING, b STRING, c int, d int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T2 +PREHOOK: query: INSERT OVERWRITE TABLE T2 +SELECT a, b, c, c from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t2 +POSTHOOK: query: INSERT OVERWRITE TABLE T2 +SELECT a, b, c, c from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2.a SIMPLE [(t1)t1.FieldSchema(name:a, type:string, comment:null), ] +POSTHOOK: Lineage: t2.b SIMPLE [(t1)t1.FieldSchema(name:b, type:string, comment:null), ] +POSTHOOK: Lineage: t2.c EXPRESSION [(t1)t1.FieldSchema(name:c, type:string, comment:null), ] +POSTHOOK: Lineage: t2.d EXPRESSION [(t1)t1.FieldSchema(name:c, type:string, comment:null), ] +PREHOOK: query: EXPLAIN +SELECT a, b, sum(c+d) from T2 group by a, b with cube +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT a, b, sum(c+d) from T2 group by a, b with cube +POSTHOOK: type: QUERY +POSTHOOK: Lineage: t2.a SIMPLE [(t1)t1.FieldSchema(name:a, type:string, comment:null), ] +POSTHOOK: Lineage: t2.b SIMPLE [(t1)t1.FieldSchema(name:b, type:string, comment:null), ] +POSTHOOK: Lineage: t2.c EXPRESSION [(t1)t1.FieldSchema(name:c, type:string, comment:null), ] +POSTHOOK: Lineage: t2.d EXPRESSION [(t1)t1.FieldSchema(name:c, type:string, comment:null), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T2))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTION sum (+ (TOK_TABLE_OR_COL c) (TOK_TABLE_OR_COL d))))) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + t2 + TableScan + alias: t2 + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + expr: c + type: int + expr: d + type: int + outputColumnNames: a, b, c, d + Group By Operator + aggregations: + expr: sum((c + d)) + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: '0' + type: string + mode: partials + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: final + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT a, b, sum(c+d) from T2 group by a, b with cube +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT a, b, sum(c+d) from T2 group by a, b with cube +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: Lineage: t2.a SIMPLE [(t1)t1.FieldSchema(name:a, type:string, comment:null), ] +POSTHOOK: Lineage: t2.b SIMPLE [(t1)t1.FieldSchema(name:b, type:string, comment:null), ] +POSTHOOK: Lineage: t2.c EXPRESSION [(t1)t1.FieldSchema(name:c, type:string, comment:null), ] +POSTHOOK: Lineage: t2.d EXPRESSION [(t1)t1.FieldSchema(name:c, type:string, comment:null), ] +NULL NULL 46 +NULL 1 8 +NULL 2 28 +NULL 3 10 +1 NULL 6 +1 1 6 +2 NULL 18 +2 2 8 +2 3 10 +3 NULL 16 +3 2 16 +5 NULL 4 +5 2 4 +8 NULL 2 +8 1 2 Index: ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out =================================================================== --- ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out (revision 0) +++ ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out (working copy) @@ -0,0 +1,433 @@ +PREHOOK: query: -- Set merging to false above to make the explain more readable + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- Set merging to false above to make the explain more readable + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1 +PREHOOK: type: LOAD +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t1 +PREHOOK: query: -- This tests that cubes and rollups work fine where the source is a sub-query +EXPLAIN +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube +PREHOOK: type: QUERY +POSTHOOK: query: -- This tests that cubes and rollups work fine where the source is a sub-query +EXPLAIN +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTION count 1))) (TOK_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + subq1:t1 + TableScan + alias: t1 + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + outputColumnNames: a, b + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: _col0 + type: string + expr: _col1 + type: string + expr: '0' + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +NULL NULL 6 +NULL 1 2 +NULL 2 3 +NULL 3 1 +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 +PREHOOK: query: -- Since 4 grouping sets would be generated for the cube, an additional MR job should be created +EXPLAIN +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube +PREHOOK: type: QUERY +POSTHOOK: query: -- Since 4 grouping sets would be generated for the cube, an additional MR job should be created +EXPLAIN +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME T1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTION count 1))) (TOK_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL b)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_CUBE_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL b)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + subq1:t1 + TableScan + alias: t1 + Select Operator + expressions: + expr: a + type: string + expr: b + type: string + outputColumnNames: a, b + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: a + type: string + expr: b + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: _col0 + type: string + expr: _col1 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: '0' + type: string + mode: partials + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + sort order: +++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + tag: -1 + value expressions: + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + expr: KEY._col2 + type: string + mode: final + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +NULL NULL 6 +NULL 1 2 +NULL 2 3 +NULL 3 1 +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 Index: ql/src/test/results/clientnegative/groupby_grouping_sets6.q.out =================================================================== --- ql/src/test/results/clientnegative/groupby_grouping_sets6.q.out (revision 0) +++ ql/src/test/results/clientnegative/groupby_grouping_sets6.q.out (working copy) @@ -0,0 +1,6 @@ +PREHOOK: query: CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T1 +FAILED: SemanticException [Error 10216]: An additional MR job is introduced since the cardinality of grouping sets is more than hive.new.job.grouping.set.cardinality. This functionality is not supported with distincts. Either set hive.new.job.grouping.set.cardinality to a high number (higher than the number of rows per input row due to grouping sets in the query), or rewrite the query to not use distincts. The number of rows per input row due to grouping sets is 4 Index: ql/src/test/results/clientnegative/groupby_grouping_sets7.q.out =================================================================== --- ql/src/test/results/clientnegative/groupby_grouping_sets7.q.out (revision 0) +++ ql/src/test/results/clientnegative/groupby_grouping_sets7.q.out (working copy) @@ -0,0 +1,6 @@ +PREHOOK: query: CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T1 +FAILED: SemanticException [Error 10215]: An additional MR job is introduced since the number of rows created per input row due to grouping sets is more than hive.new.job.grouping.set.cardinality. There is no need to handle skew separately. set hive.groupby.skewindata to false. The number of rows per input row due to grouping sets is 4 Index: ql/src/test/results/compiler/plan/groupby2.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby2.q.xml (revision 1425241) +++ ql/src/test/results/compiler/plan/groupby2.q.xml (working copy) @@ -734,7 +734,7 @@ - + 0.9 @@ -1739,6 +1739,9 @@ + + + 0.9 Index: ql/src/test/results/compiler/plan/groupby4.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby4.q.xml (revision 1425241) +++ ql/src/test/results/compiler/plan/groupby4.q.xml (working copy) @@ -449,7 +449,7 @@ - + 0.9 @@ -1197,6 +1197,9 @@ + + + 0.9 Index: ql/src/test/results/compiler/plan/groupby6.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby6.q.xml (revision 1425241) +++ ql/src/test/results/compiler/plan/groupby6.q.xml (working copy) @@ -449,7 +449,7 @@ - + 0.9 @@ -1197,6 +1197,9 @@ + + + 0.9 Index: ql/src/test/results/compiler/plan/groupby1.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby1.q.xml (revision 1425241) +++ ql/src/test/results/compiler/plan/groupby1.q.xml (working copy) @@ -657,7 +657,7 @@ - + 0.9 @@ -1498,6 +1498,9 @@ + + + 0.9 Index: ql/src/test/results/compiler/plan/groupby3.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby3.q.xml (revision 1425241) +++ ql/src/test/results/compiler/plan/groupby3.q.xml (working copy) @@ -923,7 +923,7 @@ - + 0.9 @@ -2058,6 +2058,9 @@ + + + 0.9 Index: ql/src/test/results/compiler/plan/groupby5.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby5.q.xml (revision 1425241) +++ ql/src/test/results/compiler/plan/groupby5.q.xml (working copy) @@ -504,7 +504,7 @@ - + 0.9 @@ -1366,6 +1366,9 @@ + + + 0.9 Index: ql/src/test/queries/clientpositive/groupby_grouping_sets3.q =================================================================== --- ql/src/test/queries/clientpositive/groupby_grouping_sets3.q (revision 0) +++ ql/src/test/queries/clientpositive/groupby_grouping_sets3.q (working copy) @@ -0,0 +1,27 @@ +-- In this test, 2 files are loaded into table T1. The data contains rows with the same value of a and b, +-- with different number of rows for a and b in each file. Since bucketizedHiveInputFormat is used, +-- this tests that the aggregate function stores the partial aggregate state correctly even if an +-- additional MR job is created for processing the grouping sets. +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/grouping_sets1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../data/files/grouping_sets2.txt' INTO TABLE T1; + +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.new.job.grouping.set.cardinality = 30; + +-- The query below will execute in a single MR job, since 4 rows are generated per input row +-- (cube of a,b will lead to (a,b), (a, null), (null, b) and (null, null) and +-- hive.new.job.grouping.set.cardinality is more than 4. +EXPLAIN +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube; +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube; + +set hive.new.job.grouping.set.cardinality=2; + +-- The query below will execute in 2 MR jobs, since hive.new.job.grouping.set.cardinality is set to 2. +-- The partial aggregation state should be maintained correctly across MR jobs. +EXPLAIN +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube; +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube; + Index: ql/src/test/queries/clientpositive/groupby_grouping_sets4.q =================================================================== --- ql/src/test/queries/clientpositive/groupby_grouping_sets4.q (revision 0) +++ ql/src/test/queries/clientpositive/groupby_grouping_sets4.q (working copy) @@ -0,0 +1,39 @@ +set hive.merge.mapfiles = false; +set hive.merge.mapredfiles = false; +-- Set merging to false above to make the explain more readable + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; + +-- This tests that cubes and rollups work fine inside sub-queries. +EXPLAIN +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a; + +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a; + +set hive.new.job.grouping.set.cardinality=2; + +-- Since 4 grouping sets would be generated for each sub-query, an additional MR job should be created +-- for each of them +EXPLAIN +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a; + +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a; + Index: ql/src/test/queries/clientpositive/groupby_grouping_sets5.q =================================================================== --- ql/src/test/queries/clientpositive/groupby_grouping_sets5.q (revision 0) +++ ql/src/test/queries/clientpositive/groupby_grouping_sets5.q (working copy) @@ -0,0 +1,25 @@ +set hive.merge.mapfiles = false; +set hive.merge.mapredfiles = false; +-- Set merging to false above to make the explain more readable + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; + +-- This tests that cubes and rollups work fine where the source is a sub-query +EXPLAIN +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube; + +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube; + +set hive.new.job.grouping.set.cardinality=2; + +-- Since 4 grouping sets would be generated for the cube, an additional MR job should be created +EXPLAIN +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube; + +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube; Index: ql/src/test/queries/clientpositive/groupby_grouping_sets2.q =================================================================== --- ql/src/test/queries/clientpositive/groupby_grouping_sets2.q (revision 0) +++ ql/src/test/queries/clientpositive/groupby_grouping_sets2.q (working copy) @@ -0,0 +1,23 @@ +set hive.new.job.grouping.set.cardinality=2; + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; + +-- Since 4 grouping sets would be generated for the query below, an additional MR job should be created +EXPLAIN +SELECT a, b, count(*) from T1 group by a, b with cube; +SELECT a, b, count(*) from T1 group by a, b with cube; + +EXPLAIN +SELECT a, b, sum(c) from T1 group by a, b with cube; +SELECT a, b, sum(c) from T1 group by a, b with cube; + +CREATE TABLE T2(a STRING, b STRING, c int, d int); + +INSERT OVERWRITE TABLE T2 +SELECT a, b, c, c from T1; + +EXPLAIN +SELECT a, b, sum(c+d) from T2 group by a, b with cube; +SELECT a, b, sum(c+d) from T2 group by a, b with cube; Index: ql/src/test/queries/clientnegative/groupby_grouping_sets6.q =================================================================== --- ql/src/test/queries/clientnegative/groupby_grouping_sets6.q (revision 0) +++ ql/src/test/queries/clientnegative/groupby_grouping_sets6.q (working copy) @@ -0,0 +1,8 @@ +set hive.new.job.grouping.set.cardinality=2; + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; + +-- Since 4 grouping sets would be generated for the query below, an additional MR job should be created +-- This is not allowed with distincts. +SELECT a, b, count(distinct c) from T1 group by a, b with cube; + Index: ql/src/test/queries/clientnegative/groupby_grouping_sets7.q =================================================================== --- ql/src/test/queries/clientnegative/groupby_grouping_sets7.q (revision 0) +++ ql/src/test/queries/clientnegative/groupby_grouping_sets7.q (working copy) @@ -0,0 +1,10 @@ +set hive.new.job.grouping.set.cardinality=2; +set hive.map.aggr=true; +set hive.groupby.skewindata=true; + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; + +-- Since 4 grouping sets would be generated for the query below, an additional MR job should be created +-- This is not allowed with map-side aggregation and skew +SELECT a, b, count(1) from T1 group by a, b with cube; + Index: ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java (revision 1425241) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java (working copy) @@ -214,9 +214,6 @@ HiveConf.ConfVars.HIVESENDHEARTBEAT); countAfterReport = 0; groupingSetsPresent = conf.isGroupingSetsPresent(); - groupingSets = conf.getListGroupingSets(); - groupingSetsPosition = conf.getGroupingSetPosition(); - ObjectInspector rowInspector = inputObjInspectors[0]; // init keyFields @@ -236,6 +233,8 @@ // Initialize the constants for the grouping sets, so that they can be re-used for // each row if (groupingSetsPresent) { + groupingSets = conf.getListGroupingSets(); + groupingSetsPosition = conf.getGroupingSetPosition(); newKeysGroupingSets = new ArrayList(); groupingSetsBitSet = new ArrayList(); Index: ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (revision 1425241) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (working copy) @@ -216,11 +216,11 @@ private final UnparseTranslator unparseTranslator; private final GlobalLimitCtx globalLimitCtx = new GlobalLimitCtx(); - //prefix for column names auto generated by hive + // prefix for column names auto generated by hive private final String autogenColAliasPrfxLbl; private final boolean autogenColAliasPrfxIncludeFuncName; - //Max characters when auto generating the column name with func name + // Max characters when auto generating the column name with func name private static final int AUTOGEN_COLALIAS_PRFX_MAXLENGTH = 20; private static class Phase1Ctx { @@ -249,9 +249,9 @@ prunedPartitions = new HashMap(); unparseTranslator = new UnparseTranslator(); autogenColAliasPrfxLbl = HiveConf.getVar(conf, - HiveConf.ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL); + HiveConf.ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL); autogenColAliasPrfxIncludeFuncName = HiveConf.getBoolVar(conf, - HiveConf.ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME); + HiveConf.ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME); queryProperties = new QueryProperties(); opToPartToSkewedPruner = new HashMap>(); } @@ -354,7 +354,8 @@ ASTNode selectExpr, QBParseInfo qbp) { for (int i = 0; i < selectExpr.getChildCount(); ++i) { ASTNode selExpr = (ASTNode) selectExpr.getChild(i); - if ((selExpr.getToken().getType() == HiveParser.TOK_SELEXPR) && (selExpr.getChildCount() == 2)) { + if ((selExpr.getToken().getType() == HiveParser.TOK_SELEXPR) + && (selExpr.getChildCount() == 2)) { String columnAlias = unescapeIdentifier(selExpr.getChild(1).getText()); qbp.setExprToColumnAlias((ASTNode) selExpr.getChild(0), columnAlias); } @@ -489,15 +490,15 @@ // Need to change it to list of columns if (sampleCols.size() > 2) { throw new SemanticException(generateErrorMessage( - (ASTNode) tabref.getChild(0), - ErrorMsg.SAMPLE_RESTRICTION.getMsg())); + (ASTNode) tabref.getChild(0), + ErrorMsg.SAMPLE_RESTRICTION.getMsg())); } qb.getParseInfo().setTabSample( alias, new TableSample( - unescapeIdentifier(sampleClause.getChild(0).getText()), - unescapeIdentifier(sampleClause.getChild(1).getText()), - sampleCols)); + unescapeIdentifier(sampleClause.getChild(0).getText()), + unescapeIdentifier(sampleClause.getChild(1).getText()), + sampleCols)); if (unparseTranslator.isEnabled()) { for (ASTNode sampleCol : sampleCols) { unparseTranslator.addIdentifierTranslation((ASTNode) sampleCol @@ -621,7 +622,7 @@ if ((numChildren != 2) && (numChildren != 3) && join.getToken().getType() != HiveParser.TOK_UNIQUEJOIN) { throw new SemanticException(generateErrorMessage(join, - "Join with multiple children")); + "Join with multiple children")); } for (int num = 0; num < numChildren; num++) { @@ -734,7 +735,7 @@ case HiveParser.TOK_INSERT_INTO: String currentDatabase = db.getCurrentDatabase(); - String tab_name = getUnescapedName((ASTNode)ast.getChild(0).getChild(0), currentDatabase); + String tab_name = getUnescapedName((ASTNode) ast.getChild(0).getChild(0), currentDatabase); qbp.addInsertIntoTable(tab_name); case HiveParser.TOK_DESTINATION: @@ -758,7 +759,7 @@ int child_count = ast.getChildCount(); if (child_count != 1) { throw new SemanticException(generateErrorMessage(ast, - "Multiple Children " + child_count)); + "Multiple Children " + child_count)); } // Check if this is a subquery / lateral view @@ -791,10 +792,10 @@ qbp.setDistributeByExprForClause(ctx_1.dest, ast); if (qbp.getClusterByForClause(ctx_1.dest) != null) { throw new SemanticException(generateErrorMessage(ast, - ErrorMsg.CLUSTERBY_DISTRIBUTEBY_CONFLICT.getMsg())); + ErrorMsg.CLUSTERBY_DISTRIBUTEBY_CONFLICT.getMsg())); } else if (qbp.getOrderByForClause(ctx_1.dest) != null) { throw new SemanticException(generateErrorMessage(ast, - ErrorMsg.ORDERBY_DISTRIBUTEBY_CONFLICT.getMsg())); + ErrorMsg.ORDERBY_DISTRIBUTEBY_CONFLICT.getMsg())); } break; @@ -805,10 +806,10 @@ qbp.setSortByExprForClause(ctx_1.dest, ast); if (qbp.getClusterByForClause(ctx_1.dest) != null) { throw new SemanticException(generateErrorMessage(ast, - ErrorMsg.CLUSTERBY_SORTBY_CONFLICT.getMsg())); + ErrorMsg.CLUSTERBY_SORTBY_CONFLICT.getMsg())); } else if (qbp.getOrderByForClause(ctx_1.dest) != null) { throw new SemanticException(generateErrorMessage(ast, - ErrorMsg.ORDERBY_SORTBY_CONFLICT.getMsg())); + ErrorMsg.ORDERBY_SORTBY_CONFLICT.getMsg())); } break; @@ -820,7 +821,7 @@ qbp.setOrderByExprForClause(ctx_1.dest, ast); if (qbp.getClusterByForClause(ctx_1.dest) != null) { throw new SemanticException(generateErrorMessage(ast, - ErrorMsg.CLUSTERBY_ORDERBY_CONFLICT.getMsg())); + ErrorMsg.CLUSTERBY_ORDERBY_CONFLICT.getMsg())); } break; @@ -836,7 +837,7 @@ } if (qbp.getSelForClause(ctx_1.dest).getToken().getType() == HiveParser.TOK_SELECTDI) { throw new SemanticException(generateErrorMessage(ast, - ErrorMsg.SELECT_DISTINCT_WITH_GROUPBY.getMsg())); + ErrorMsg.SELECT_DISTINCT_WITH_GROUPBY.getMsg())); } qbp.setGroupByExprForClause(ctx_1.dest, ast); skipRecursion = true; @@ -863,7 +864,7 @@ case HiveParser.TOK_ANALYZE: // Case of analyze command - String table_name = getUnescapedName((ASTNode)ast.getChild(0).getChild(0)); + String table_name = getUnescapedName((ASTNode) ast.getChild(0).getChild(0)); qb.setTabAlias(table_name, table_name); @@ -881,7 +882,7 @@ // select * from (subq1 union subq2) subqalias if (!qbp.getIsSubQ()) { throw new SemanticException(generateErrorMessage(ast, - ErrorMsg.UNION_NOTIN_SUBQ.getMsg())); + ErrorMsg.UNION_NOTIN_SUBQ.getMsg())); } case HiveParser.TOK_INSERT: @@ -924,7 +925,7 @@ } } else { throw new SemanticException(ErrorMsg.INSERT_INTO_DYNAMICPARTITION_IFNOTEXISTS - .getMsg(partition.toString())); + .getMsg(partition.toString())); } } @@ -976,7 +977,7 @@ } // Disallow INSERT INTO on bucketized tables - if(qb.getParseInfo().isInsertIntoTable(tab.getDbName(), tab.getTableName()) && + if (qb.getParseInfo().isInsertIntoTable(tab.getDbName(), tab.getTableName()) && tab.getNumBuckets() > 0) { throw new SemanticException(ErrorMsg.INSERT_INTO_BUCKETIZED_TABLE. getMsg("Table: " + tab_name)); @@ -998,9 +999,9 @@ if (qb.getParseInfo().isAnalyzeCommand()) { throw new SemanticException(ErrorMsg.ANALYZE_VIEW.getMsg()); } - String fullViewName = tab.getDbName()+"."+tab.getTableName(); + String fullViewName = tab.getDbName() + "." + tab.getTableName(); // Prevent view cycles - if(viewsExpanded.contains(fullViewName)){ + if (viewsExpanded.contains(fullViewName)) { throw new SemanticException("Recursive view " + fullViewName + " detected (cycle: " + StringUtils.join(viewsExpanded, " -> ") + " -> " + fullViewName + ")."); @@ -1015,8 +1016,8 @@ if (!InputFormat.class.isAssignableFrom(tab.getInputFormatClass())) { throw new SemanticException(generateErrorMessage( - qb.getParseInfo().getSrcForAlias(alias), - ErrorMsg.INVALID_INPUT_FORMAT_TYPE.getMsg())); + qb.getParseInfo().getSrcForAlias(alias), + ErrorMsg.INVALID_INPUT_FORMAT_TYPE.getMsg())); } qb.getMetaData().setSrcForAlias(alias, tab); @@ -1027,8 +1028,9 @@ try { ts.partitions = db.getPartitionsByNames(ts.tableHandle, ts.partSpec); } catch (HiveException e) { - throw new SemanticException(generateErrorMessage(qb.getParseInfo().getSrcForAlias(alias), - "Cannot get partitions for " + ts.partSpec), e); + throw new SemanticException(generateErrorMessage( + qb.getParseInfo().getSrcForAlias(alias), + "Cannot get partitions for " + ts.partSpec), e); } } qb.getParseInfo().addTableSpec(alias, ts); @@ -1045,7 +1047,7 @@ QBExpr qbexpr = qb.getSubqForAlias(alias); getMetaData(qbexpr); if (wasView) { - viewsExpanded.remove(viewsExpanded.size()-1); + viewsExpanded.remove(viewsExpanded.size() - 1); } } @@ -1105,7 +1107,7 @@ ctx.setResFile(null); // allocate a temporary output dir on the location of the table - String tableName = getUnescapedName((ASTNode)ast.getChild(0)); + String tableName = getUnescapedName((ASTNode) ast.getChild(0)); Table newTable = db.newTable(tableName); Path location; try { @@ -1119,7 +1121,7 @@ FileUtils.makeQualified(location, conf).toUri()); } catch (Exception e) { throw new SemanticException(generateErrorMessage(ast, - "Error creating temporary folder on: " + location.toString()), e); + "Error creating temporary folder on: " + location.toString()), e); } if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVESTATSAUTOGATHER)) { tableSpec ts = new tableSpec(db, conf, this.ast); @@ -1140,7 +1142,7 @@ } default: throw new SemanticException(generateErrorMessage(ast, - "Unknown Token Type " + ast.getToken().getType())); + "Unknown Token Type " + ast.getToken().getType())); } } } catch (HiveException e) { @@ -1158,7 +1160,7 @@ ASTNode viewTree; final ASTNodeOrigin viewOrigin = new ASTNodeOrigin("VIEW", tab.getTableName(), tab.getViewExpandedText(), alias, qb.getParseInfo().getSrcForAlias( - alias)); + alias)); try { String viewText = tab.getViewExpandedText(); // Reparse text, passing null for context to avoid clobbering @@ -1340,6 +1342,7 @@ joinTree.addFilterMapping(cond.getRight(), cond.getLeft(), filters.get(1).size()); } } + /** * Parse the join condition. If the condition is a join condition, throw an * error if it is not an equality. Otherwise, break it into left and right @@ -1405,7 +1408,7 @@ if ((rightCondAl1.size() != 0) || ((rightCondAl1.size() == 0) && (rightCondAl2.size() == 0))) { if (type.equals(JoinType.LEFTOUTER) || - type.equals(JoinType.FULLOUTER)) { + type.equals(JoinType.FULLOUTER)) { if (conf.getBoolVar(HiveConf.ConfVars.HIVEOUTERJOINSUPPORTSFILTERS)) { joinTree.getFilters().get(0).add(joinCond); } else { @@ -1492,7 +1495,7 @@ for (int ci = childrenBegin; ci < joinCond.getChildCount(); ci++) { parseJoinCondPopulateAlias(joinTree, (ASTNode) joinCond.getChild(ci), leftAlias.get(ci - childrenBegin), rightAlias.get(ci - - childrenBegin), null); + - childrenBegin), null); } boolean leftAliasNull = true; @@ -1573,7 +1576,7 @@ Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new FilterDesc(genExprNodeDesc(condn, inputRR), false), new RowSchema( - inputRR.getColumnInfos()), input), inputRR); + inputRR.getColumnInfos()), input), inputRR); return output; } @@ -1604,11 +1607,11 @@ RowResolver inputRR = inputCtx.getRowResolver(); Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new FilterDesc(genExprNodeDesc(condn, inputRR), false), new RowSchema( - inputRR.getColumnInfos()), input), inputRR); + inputRR.getColumnInfos()), input), inputRR); if (LOG.isDebugEnabled()) { LOG.debug("Created Filter Plan for " + qb.getId() + " row schema: " - + inputRR.toString()); + + inputRR.toString()); } return output; } @@ -1676,8 +1679,8 @@ col_list.add(expr); output.put(tmp[0], tmp[1], new ColumnInfo(getColumnInternalName(pos), colInfo.getType(), - colInfo.getTabAlias(), colInfo.getIsVirtualCol(), - colInfo.isHiddenVirtualCol())); + colInfo.getTabAlias(), colInfo.getIsVirtualCol(), + colInfo.isHiddenVirtualCol())); pos = Integer.valueOf(pos.intValue() + 1); matched++; @@ -1799,7 +1802,7 @@ tblDesc.getProperties().setProperty(serdeConstants.LINE_DELIM, lineDelim); if (!lineDelim.equals("\n") && !lineDelim.equals("10")) { throw new SemanticException(generateErrorMessage(rowChild, - ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg())); + ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg())); } break; default: @@ -1884,7 +1887,7 @@ String intName = getColumnInternalName(i); ColumnInfo colInfo = new ColumnInfo(intName, TypeInfoUtils .getTypeInfoFromTypeString(getTypeStringFromAST((ASTNode) child - .getChild(1))), null, false); + .getChild(1))), null, false); colInfo.setAlias(colAlias); outputCols.add(colInfo); } @@ -1977,8 +1980,8 @@ Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new ScriptDesc( - fetchFilesNotInLocalFilesystem(stripQuotes(trfm.getChild(execPos).getText())), - inInfo, inRecordWriter, outInfo, outRecordReader, errRecordReader, errInfo), + fetchFilesNotInLocalFilesystem(stripQuotes(trfm.getChild(execPos).getText())), + inInfo, inRecordWriter, outInfo, outRecordReader, errRecordReader, errInfo), new RowSchema(out_rwsch.getColumnInfos()), input), out_rwsch); return output; @@ -2037,7 +2040,7 @@ private List getGroupingSetsForRollup(int size) { List groupingSetKeys = new ArrayList(); for (int i = 0; i <= size; i++) { - groupingSetKeys.add((1 << i) - 1); + groupingSetKeys.add((1 << i) - 1); } return groupingSetKeys; } @@ -2045,7 +2048,7 @@ private List getGroupingSetsForCube(int size) { int count = 1 << size; List results = new ArrayList(count); - for(int i = 0; i < count; ++i) { + for (int i = 0; i < count; ++i) { results.add(i); } return results; @@ -2082,16 +2085,16 @@ if (root != null) { for (int i = 0; i < root.getChildCount(); ++i) { ASTNode child = (ASTNode) root.getChild(i); - if(child.getType() != HiveParser.TOK_GROUPING_SETS_EXPRESSION) { + if (child.getType() != HiveParser.TOK_GROUPING_SETS_EXPRESSION) { continue; } int bitmap = 0; for (int j = 0; j < child.getChildCount(); ++j) { String treeAsString = child.getChild(j).toStringTree(); Integer pos = exprPos.get(treeAsString); - if(pos == null) { + if (pos == null) { throw new SemanticException( - generateErrorMessage((ASTNode)child.getChild(j), + generateErrorMessage((ASTNode) child.getChild(j), ErrorMsg.HIVE_GROUPING_SETS_EXPR_NOT_IN_GROUPBY.getErrorCodedMsg())); } bitmap = setBit(bitmap, pos); @@ -2099,7 +2102,7 @@ result.add(bitmap); } } - if(checkForNoAggr(result)) { + if (checkForNoAggr(result)) { throw new SemanticException( ErrorMsg.HIVE_GROUPING_SETS_AGGR_NOFUNC.getMsg()); } @@ -2108,7 +2111,7 @@ private boolean checkForNoAggr(List bitmaps) { boolean ret = true; - for(int mask : bitmaps) { + for (int mask : bitmaps) { ret &= mask == 0; } return ret; @@ -2172,7 +2175,7 @@ ASTNode root = (ASTNode) selExpr.getChild(0); if (root.getType() == HiveParser.TOK_TABLE_OR_COL) { colAlias = - BaseSemanticAnalyzer.unescapeIdentifier(root.getChild(0).getText()); + BaseSemanticAnalyzer.unescapeIdentifier(root.getChild(0).getText()); colRef[0] = tabAlias; colRef[1] = colAlias; return colRef; @@ -2194,23 +2197,23 @@ } } - //if specified generate alias using func name + // if specified generate alias using func name if (includeFuncName && (root.getType() == HiveParser.TOK_FUNCTION)) { String expr_flattened = root.toStringTree(); - //remove all TOK tokens + // remove all TOK tokens String expr_no_tok = expr_flattened.replaceAll("TOK_\\S+", ""); - //remove all non alphanumeric letters, replace whitespace spans with underscore - String expr_formatted = expr_no_tok.replaceAll("\\W", " ").trim().replaceAll("\\s+", "_"); + // remove all non alphanumeric letters, replace whitespace spans with underscore + String expr_formatted = expr_no_tok.replaceAll("\\W", " ").trim().replaceAll("\\s+", "_"); - //limit length to 20 chars - if(expr_formatted.length()>AUTOGEN_COLALIAS_PRFX_MAXLENGTH) { + // limit length to 20 chars + if (expr_formatted.length() > AUTOGEN_COLALIAS_PRFX_MAXLENGTH) { expr_formatted = expr_formatted.substring(0, AUTOGEN_COLALIAS_PRFX_MAXLENGTH); } - //append colnum to make it unique + // append colnum to make it unique colAlias = expr_formatted.concat("_" + colNum); } @@ -2273,7 +2276,7 @@ } boolean isInTransform = (selExprList.getChild(posn).getChild(0).getType() == - HiveParser.TOK_TRANSFORM); + HiveParser.TOK_TRANSFORM); if (isInTransform) { queryProperties.setUsesScript(true); globalLimitCtx.setHasTransformOrUDTF(true); @@ -2313,14 +2316,14 @@ // Only support a single expression when it's a UDTF if (selExprList.getChildCount() > 1) { throw new SemanticException(generateErrorMessage( - (ASTNode) selExprList.getChild(1), - ErrorMsg.UDTF_MULTIPLE_EXPR.getMsg())); + (ASTNode) selExprList.getChild(1), + ErrorMsg.UDTF_MULTIPLE_EXPR.getMsg())); } // Require an AS for UDTFs for column aliases ASTNode selExpr = (ASTNode) selExprList.getChild(posn); if (selExpr.getChildCount() < 2) { throw new SemanticException(generateErrorMessage(udtfExpr, - ErrorMsg.UDTF_REQUIRE_AS.getMsg())); + ErrorMsg.UDTF_REQUIRE_AS.getMsg())); } // Get the column / table aliases from the expression. Start from 1 as // 0 is the TOK_FUNCTION @@ -2381,8 +2384,8 @@ // AST's are slightly different. if (!isInTransform && !isUDTF && child.getChildCount() > 2) { throw new SemanticException(generateErrorMessage( - (ASTNode) child.getChild(2), - ErrorMsg.INVALID_AS.getMsg())); + (ASTNode) child.getChild(2), + ErrorMsg.INVALID_AS.getMsg())); } // The real expression @@ -2398,7 +2401,7 @@ // Get rid of TOK_SELEXPR expr = (ASTNode) child.getChild(0); String[] colRef = getColAlias(child, autogenColAliasPrfxLbl, inputRR, - autogenColAliasPrfxIncludeFuncName, i); + autogenColAliasPrfxIncludeFuncName, i); tabAlias = colRef[0]; colAlias = colRef[1]; if (hasAsClause) { @@ -2451,9 +2454,9 @@ } ColumnInfo colInfo = new ColumnInfo(getColumnInternalName(pos), - exp.getWritableObjectInspector(), tabAlias, false); + exp.getWritableObjectInspector(), tabAlias, false); colInfo.setSkewedCol((exp instanceof ExprNodeColumnDesc) ? ((ExprNodeColumnDesc) exp) - .isSkewedCol() : false); + .isSkewedCol() : false); out_rwsch.put(tabAlias, colAlias, colInfo); pos = Integer.valueOf(pos.intValue() + 1); @@ -2476,7 +2479,7 @@ Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new SelectDesc(col_list, columnNames, selectStar), new RowSchema( - out_rwsch.getColumnInfos()), input), out_rwsch); + out_rwsch.getColumnInfos()), input), out_rwsch); output.setColumnExprMap(colExprMap); if (isInTransform) { @@ -2556,7 +2559,7 @@ boolean isDistinct, boolean isAllColumns) throws SemanticException { ArrayList originalParameterTypeInfos = - getWritableObjectInspector(aggParameters); + getWritableObjectInspector(aggParameters); GenericUDAFEvaluator result = FunctionRegistry.getGenericUDAFEvaluator( aggName, originalParameterTypeInfos, isDistinct, isAllColumns); if (null == result) { @@ -2732,9 +2735,9 @@ List inputKeyCols = ((ReduceSinkDesc) reduceSinkOperatorInfo.getConf()).getOutputKeyColumnNames(); if (inputKeyCols.size() > 0) { - lastKeyColName = inputKeyCols.get(inputKeyCols.size()-1); + lastKeyColName = inputKeyCols.get(inputKeyCols.size() - 1); } - reduceValues = ((ReduceSinkDesc)reduceSinkOperatorInfo.getConf()).getValueCols(); + reduceValues = ((ReduceSinkDesc) reduceSinkOperatorInfo.getConf()).getValueCols(); } int numDistinctUDFs = 0; for (Map.Entry entry : aggregationTrees.entrySet()) { @@ -2751,7 +2754,7 @@ for (int i = 1; i < value.getChildCount(); i++) { ASTNode paraExpr = (ASTNode) value.getChild(i); ColumnInfo paraExprInfo = - groupByInputRowResolver.getExpression(paraExpr); + groupByInputRowResolver.getExpression(paraExpr); if (paraExprInfo == null) { throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(paraExpr)); } @@ -2762,8 +2765,8 @@ // if aggr is distinct, the parameter is name is constructed as // KEY.lastKeyColName:._colx paraExpression = Utilities.ReduceField.KEY.name() + "." + - lastKeyColName + ":" + numDistinctUDFs + "." + - getColumnInternalName(i-1); + lastKeyColName + ":" + numDistinctUDFs + "." + + getColumnInternalName(i - 1); } @@ -2805,13 +2808,14 @@ } } float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); - float memoryThreshold = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + float memoryThreshold = HiveConf + .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( - new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, - false,groupByMemoryUsage,memoryThreshold, null, false, 0), - new RowSchema(groupByOutputRowResolver.getColumnInfos()), - reduceSinkOperatorInfo), groupByOutputRowResolver); + new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, + false, groupByMemoryUsage, memoryThreshold, null, false, 0), + new RowSchema(groupByOutputRowResolver.getColumnInfos()), + reduceSinkOperatorInfo), groupByOutputRowResolver); op.setColumnExprMap(colExprMap); return op; } @@ -2822,7 +2826,7 @@ // For eg: consider: select key, value, count(1) from T group by key, value with rollup. // Assuming map-side aggregation and no skew, the plan would look like: // - // TableScan --> Select --> GroupBy1 --> ReduceSink --> GroupBy2 --> Select --> FileSink + // TableScan --> Select --> GroupBy1 --> ReduceSink --> GroupBy2 --> Select --> FileSink // // This function is called for GroupBy2 to pass the additional grouping keys introduced by // GroupBy1 for the grouping set (corresponding to the rollup). @@ -2833,9 +2837,9 @@ Map colExprMap) throws SemanticException { // For grouping sets, add a dummy grouping key String groupingSetColumnName = - groupByInputRowResolver.get(null, VirtualColumn.GROUPINGID.getName()).getInternalName(); + groupByInputRowResolver.get(null, VirtualColumn.GROUPINGID.getName()).getInternalName(); ExprNodeDesc inputExpr = new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, - groupingSetColumnName, null, false); + groupingSetColumnName, null, false); groupByKeys.add(inputExpr); String field = getColumnInternalName(groupByKeys.size() - 1); @@ -2853,7 +2857,7 @@ // For eg: consider: select key, value, count(1) from T group by key, value with rollup. // Assuming map-side aggregation and no skew, the plan would look like: // - // TableScan --> Select --> GroupBy1 --> ReduceSink --> GroupBy2 --> Select --> FileSink + // TableScan --> Select --> GroupBy1 --> ReduceSink --> GroupBy2 --> Select --> FileSink // // This function is called for ReduceSink to add the additional grouping keys introduced by // GroupBy1 into the reduce keys. @@ -2864,16 +2868,16 @@ Map colExprMap) throws SemanticException { // add a key for reduce sink String groupingSetColumnName = - reduceSinkInputRowResolver.get(null, VirtualColumn.GROUPINGID.getName()).getInternalName(); + reduceSinkInputRowResolver.get(null, VirtualColumn.GROUPINGID.getName()).getInternalName(); ExprNodeDesc inputExpr = new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, - groupingSetColumnName, null, false); + groupingSetColumnName, null, false); reduceKeys.add(inputExpr); outputKeyColumnNames.add(getColumnInternalName(reduceKeys.size() - 1)); String field = Utilities.ReduceField.KEY.toString() + "." - + getColumnInternalName(reduceKeys.size() - 1); + + getColumnInternalName(reduceKeys.size() - 1); ColumnInfo colInfo = new ColumnInfo(field, reduceKeys.get( - reduceKeys.size() - 1).getTypeInfo(), null, true); + reduceKeys.size() - 1).getTypeInfo(), null, true); reduceSinkOutputRowResolver.put(null, VirtualColumn.GROUPINGID.getName(), colInfo); colExprMap.put(colInfo.getInternalName(), inputExpr); } @@ -2890,6 +2894,12 @@ * genericUDAFEvaluator. * @param distPartAggr * partial aggregation for distincts + * @param groupingSets + * list of grouping sets + * @param groupingSetsPresent + * whether grouping sets are present in this query + * @param groupingSetsConsumedCurrentMR + * whether grouping sets are consumed by this group by * @return the new GroupByOperator */ @SuppressWarnings("nls") @@ -2897,7 +2907,9 @@ String dest, Operator reduceSinkOperatorInfo, GroupByDesc.Mode mode, Map genericUDAFEvaluators, boolean distPartAgg, - boolean groupingSetsPresent) throws SemanticException { + List groupingSets, + boolean groupingSetsPresent, + boolean groupingSetsNeedAdditionalMRJob) throws SemanticException { ArrayList outputColumnNames = new ArrayList(); RowResolver groupByInputRowResolver = opParseCtx .get(reduceSinkOperatorInfo).getRowResolver(); @@ -2925,14 +2937,39 @@ colExprMap.put(field, groupByKeys.get(groupByKeys.size() - 1)); } + // This is only needed if a new grouping set key is being created + int groupingSetsPosition = 0; + // For grouping sets, add a dummy grouping key if (groupingSetsPresent) { - addGroupingSetKey( - groupByKeys, - groupByInputRowResolver, - groupByOutputRowResolver, - outputColumnNames, - colExprMap); + // Consider the query: select a,b, count(1) from T group by a,b with cube; + // where it is being executed in a single map-reduce job + // The plan is TableScan -> GroupBy1 -> ReduceSink -> GroupBy2 -> FileSink + // GroupBy1 already added the grouping id as part of the row + // This function is called for GroupBy2 to add grouping id as part of the groupby keys + if (!groupingSetsNeedAdditionalMRJob) { + addGroupingSetKey( + groupByKeys, + groupByInputRowResolver, + groupByOutputRowResolver, + outputColumnNames, + colExprMap); + } + else { + groupingSetsPosition = groupByKeys.size(); + // The grouping set has not yet been processed. Create a new grouping key + // Consider the query: select a,b, count(1) from T group by a,b with cube; + // where it is being executed in 2 map-reduce jobs + // The plan for 1st MR is TableScan -> GroupBy1 -> ReduceSink -> GroupBy2 -> FileSink + // GroupBy1/ReduceSink worked as if grouping sets were not present + // This function is called for GroupBy2 to create new rows for grouping sets + // For each input row (a,b), 4 rows are created for the example above: + // (a,b), (a,null), (null, b), (null, null) + createNewGroupingKey(groupByKeys, + outputColumnNames, + groupByOutputRowResolver, + colExprMap); + } } HashMap aggregationTrees = parseInfo @@ -2945,9 +2982,9 @@ List inputKeyCols = ((ReduceSinkDesc) reduceSinkOperatorInfo.getConf()).getOutputKeyColumnNames(); if (inputKeyCols.size() > 0) { - lastKeyColName = inputKeyCols.get(inputKeyCols.size()-1); + lastKeyColName = inputKeyCols.get(inputKeyCols.size() - 1); } - reduceValues = ((ReduceSinkDesc)reduceSinkOperatorInfo.getConf()).getValueCols(); + reduceValues = ((ReduceSinkDesc) reduceSinkOperatorInfo.getConf()).getValueCols(); } int numDistinctUDFs = 0; for (Map.Entry entry : aggregationTrees.entrySet()) { @@ -2972,7 +3009,7 @@ for (int i = 1; i < value.getChildCount(); i++) { ASTNode paraExpr = (ASTNode) value.getChild(i); ColumnInfo paraExprInfo = - groupByInputRowResolver.getExpression(paraExpr); + groupByInputRowResolver.getExpression(paraExpr); if (paraExprInfo == null) { throw new SemanticException(ErrorMsg.INVALID_COLUMN .getMsg(paraExpr)); @@ -2984,8 +3021,8 @@ // if aggr is distinct, the parameter is name is constructed as // KEY.lastKeyColName:._colx paraExpression = Utilities.ReduceField.KEY.name() + "." + - lastKeyColName + ":" + numDistinctUDFs + "." - + getColumnInternalName(i-1); + lastKeyColName + ":" + numDistinctUDFs + "." + + getColumnInternalName(i - 1); } @@ -3012,7 +3049,7 @@ assert (paraExpression != null); aggParameters.add(new ExprNodeColumnDesc(paraExprInfo.getType(), paraExpression, paraExprInfo.getTabAlias(), paraExprInfo - .getIsVirtualCol())); + .getIsVirtualCol())); } if (isDistinct) { numDistinctUDFs++; @@ -3043,20 +3080,50 @@ field, udaf.returnType, "", false)); } float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); - float memoryThreshold = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + float memoryThreshold = HiveConf + .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); - // Nothing special needs to be done for grouping sets. - // This is the final group by operator, so multiple rows corresponding to the + // Nothing special needs to be done for grouping sets if + // this is the final group by operator, and multiple rows corresponding to the // grouping sets have been generated upstream. + // However, if an addition MR job has been created to handle grouping sets, + // additional rows corresponding to grouping sets need to be created here. Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( - new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, - distPartAgg,groupByMemoryUsage,memoryThreshold, null, false, 0), - new RowSchema(groupByOutputRowResolver.getColumnInfos()), reduceSinkOperatorInfo), - groupByOutputRowResolver); + new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, + distPartAgg, groupByMemoryUsage, memoryThreshold, + groupingSets, + groupingSetsPresent && groupingSetsNeedAdditionalMRJob, + groupingSetsPosition), + new RowSchema(groupByOutputRowResolver.getColumnInfos()), reduceSinkOperatorInfo), + groupByOutputRowResolver); op.setColumnExprMap(colExprMap); return op; } + /* + * Create a new grouping key for grouping id. + * A dummy grouping id. is added. At runtime, the group by operator + * creates 'n' rows per input row, where 'n' is the number of grouping sets. + */ + private void createNewGroupingKey(List groupByKeys, + List outputColumnNames, + RowResolver groupByOutputRowResolver, + Map colExprMap) { + // The value for the constant does not matter. It is replaced by the grouping set + // value for the actual implementation + ExprNodeConstantDesc constant = new ExprNodeConstantDesc("0"); + groupByKeys.add(constant); + String field = getColumnInternalName(groupByKeys.size() - 1); + outputColumnNames.add(field); + groupByOutputRowResolver.put(null, VirtualColumn.GROUPINGID.getName(), + new ColumnInfo( + field, + TypeInfoFactory.stringTypeInfo, + null, + true)); + colExprMap.put(field, constant); + } + /** * Generate the map-side GroupByOperator for the Query Block * (qb.getParseInfo().getXXX(dest)). The new GroupByOperator will be a child @@ -3092,13 +3159,13 @@ for (int i = 0; i < grpByExprs.size(); ++i) { ASTNode grpbyExpr = grpByExprs.get(i); ExprNodeDesc grpByExprNode = genExprNodeDesc(grpbyExpr, - groupByInputRowResolver); + groupByInputRowResolver); groupByKeys.add(grpByExprNode); String field = getColumnInternalName(i); outputColumnNames.add(field); groupByOutputRowResolver.putExpression(grpbyExpr, - new ColumnInfo(field, grpByExprNode.getTypeInfo(), "", false)); + new ColumnInfo(field, grpByExprNode.getTypeInfo(), "", false)); colExprMap.put(field, groupByKeys.get(groupByKeys.size() - 1)); } @@ -3110,48 +3177,38 @@ // For eg: consider: select key, value, count(1) from T group by key, value with rollup. // Assuming map-side aggregation and no skew, the plan would look like: // - // TableScan --> Select --> GroupBy1 --> ReduceSink --> GroupBy2 --> Select --> FileSink + // TableScan --> Select --> GroupBy1 --> ReduceSink --> GroupBy2 --> Select --> FileSink // // This function is called for GroupBy1 to create an additional grouping key // for the grouping set (corresponding to the rollup). if (groupingSetsPresent) { - // The value for the constant does not matter. It is replaced by the grouping set - // value for the actual implementation - ExprNodeConstantDesc constant = new ExprNodeConstantDesc("0"); - groupByKeys.add(constant); - String field = getColumnInternalName(groupByKeys.size() - 1); - outputColumnNames.add(field); - groupByOutputRowResolver.put(null, VirtualColumn.GROUPINGID.getName(), - new ColumnInfo( - field, - TypeInfoFactory.stringTypeInfo, - null, - true)); - colExprMap.put(field, constant); + createNewGroupingKey(groupByKeys, + outputColumnNames, + groupByOutputRowResolver, + colExprMap); } // If there is a distinctFuncExp, add all parameters to the reduceKeys. if (!parseInfo.getDistinctFuncExprsForClause(dest).isEmpty()) { List list = parseInfo.getDistinctFuncExprsForClause(dest); - for(ASTNode value: list) { + for (ASTNode value : list) { // 0 is function name for (int i = 1; i < value.getChildCount(); i++) { ASTNode parameter = (ASTNode) value.getChild(i); if (groupByOutputRowResolver.getExpression(parameter) == null) { ExprNodeDesc distExprNode = genExprNodeDesc(parameter, - groupByInputRowResolver); + groupByInputRowResolver); groupByKeys.add(distExprNode); - String field = getColumnInternalName(groupByKeys.size()-1); + String field = getColumnInternalName(groupByKeys.size() - 1); outputColumnNames.add(field); groupByOutputRowResolver.putExpression(parameter, new ColumnInfo( - field, distExprNode.getTypeInfo(), "", false)); + field, distExprNode.getTypeInfo(), "", false)); colExprMap.put(field, groupByKeys.get(groupByKeys.size() - 1)); } } } } - // For each aggregation HashMap aggregationTrees = parseInfo .getAggregationExprsForClause(dest); @@ -3166,7 +3223,7 @@ for (int i = 1; i < value.getChildCount(); i++) { ASTNode paraExpr = (ASTNode) value.getChild(i); ExprNodeDesc paraExprNode = genExprNodeDesc(paraExpr, - groupByInputRowResolver); + groupByInputRowResolver); aggParameters.add(paraExprNode); } @@ -3195,13 +3252,14 @@ } } float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); - float memoryThreshold = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + float memoryThreshold = HiveConf + .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( - new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, - false,groupByMemoryUsage,memoryThreshold, - groupingSetKeys, groupingSetsPresent, groupingSetsPosition), - new RowSchema(groupByOutputRowResolver.getColumnInfos()), - inputOperatorInfo), groupByOutputRowResolver); + new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, + false, groupByMemoryUsage, memoryThreshold, + groupingSetKeys, groupingSetsPresent, groupingSetsPosition), + new RowSchema(groupByOutputRowResolver.getColumnInfos()), + inputOperatorInfo), groupByOutputRowResolver); op.setColumnExprMap(colExprMap); return op; } @@ -3243,18 +3301,18 @@ List outputValueColumnNames = new ArrayList(); ArrayList reduceKeys = getReduceKeysForReduceSink(grpByExprs, dest, - reduceSinkInputRowResolver, reduceSinkOutputRowResolver, outputKeyColumnNames, - colExprMap); + reduceSinkInputRowResolver, reduceSinkOutputRowResolver, outputKeyColumnNames, + colExprMap); // add a key for reduce sink if (groupingSetsPresent) { // Process grouping set for the reduce sink operator processGroupingSetReduceSinkOperator( - reduceSinkInputRowResolver, - reduceSinkOutputRowResolver, - reduceKeys, - outputKeyColumnNames, - colExprMap); + reduceSinkInputRowResolver, + reduceSinkOutputRowResolver, + reduceKeys, + outputKeyColumnNames, + colExprMap); if (changeNumPartitionFields) { numPartitionFields++; @@ -3291,14 +3349,14 @@ } ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap( - OperatorFactory.getAndMakeChild( - PlanUtils.getReduceSinkDesc(reduceKeys, - groupingSetsPresent ? grpByExprs.size() + 1 : grpByExprs.size(), - reduceValues, distinctColIndices, - outputKeyColumnNames, outputValueColumnNames, true, -1, numPartitionFields, - numReducers), - new RowSchema(reduceSinkOutputRowResolver.getColumnInfos()), inputOperatorInfo), - reduceSinkOutputRowResolver); + OperatorFactory.getAndMakeChild( + PlanUtils.getReduceSinkDesc(reduceKeys, + groupingSetsPresent ? grpByExprs.size() + 1 : grpByExprs.size(), + reduceValues, distinctColIndices, + outputKeyColumnNames, outputValueColumnNames, true, -1, numPartitionFields, + numReducers), + new RowSchema(reduceSinkOutputRowResolver.getColumnInfos()), inputOperatorInfo), + reduceSinkOutputRowResolver); rsOp.setColumnExprMap(colExprMap); return rsOp; } @@ -3313,7 +3371,7 @@ for (int i = 0; i < grpByExprs.size(); ++i) { ASTNode grpbyExpr = grpByExprs.get(i); ExprNodeDesc inputExpr = genExprNodeDesc(grpbyExpr, - reduceSinkInputRowResolver); + reduceSinkInputRowResolver); reduceKeys.add(inputExpr); if (reduceSinkOutputRowResolver.getExpression(grpbyExpr) == null) { outputKeyColumnNames.add(getColumnInternalName(reduceKeys.size() - 1)); @@ -3332,7 +3390,8 @@ return reduceKeys; } - private List> getDistinctColIndicesForReduceSink(QBParseInfo parseInfo, String dest, + private List> getDistinctColIndicesForReduceSink(QBParseInfo parseInfo, + String dest, List reduceKeys, RowResolver reduceSinkInputRowResolver, RowResolver reduceSinkOutputRowResolver, List outputKeyColumnNames) throws SemanticException { @@ -3368,8 +3427,8 @@ distinctIndices.add(ri); String name = getColumnInternalName(numExprs); String field = Utilities.ReduceField.KEY.toString() + "." + colName - + ":" + i - + "." + name; + + ":" + i + + "." + name; ColumnInfo colInfo = new ColumnInfo(field, expr.getTypeInfo(), null, false); reduceSinkOutputRowResolver.putExpression(parameter, colInfo); numExprs++; @@ -3469,10 +3528,10 @@ ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap( OperatorFactory.getAndMakeChild(PlanUtils.getReduceSinkDesc(reduceKeys, - grpByExprs.size(), reduceValues, distinctColIndices, - outputKeyColumnNames, outputValueColumnNames, true, -1, grpByExprs.size(), - -1), new RowSchema(reduceSinkOutputRowResolver - .getColumnInfos()), inputOperatorInfo), reduceSinkOutputRowResolver); + grpByExprs.size(), reduceValues, distinctColIndices, + outputKeyColumnNames, outputValueColumnNames, true, -1, grpByExprs.size(), + -1), new RowSchema(reduceSinkOutputRowResolver + .getColumnInfos()), inputOperatorInfo), reduceSinkOutputRowResolver); rsOp.setColumnExprMap(colExprMap); return rsOp; } @@ -3493,10 +3552,10 @@ return nodes; } for (int i = 0; i < node.getChildCount(); i++) { - ASTNode child = (ASTNode)node.getChild(i); + ASTNode child = (ASTNode) node.getChild(i); if (child.getType() == HiveParser.TOK_TABLE_OR_COL && child.getChild(0) != null && inputRR.get(null, - BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText())) != null) { + BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText())) != null) { nodes.add(child); } else { nodes.addAll(getColumnExprsFromASTNode(child, inputRR)); @@ -3557,11 +3616,11 @@ // Note that partitioning fields dont need to change, since it is either // partitioned randomly, or by all grouping keys + distinct keys processGroupingSetReduceSinkOperator( - reduceSinkInputRowResolver2, - reduceSinkOutputRowResolver2, - reduceKeys, - outputColumnNames, - colExprMap); + reduceSinkInputRowResolver2, + reduceSinkOutputRowResolver2, + reduceKeys, + outputColumnNames, + colExprMap); } // Get partial aggregation results and store in reduceValues @@ -3585,9 +3644,9 @@ ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap( OperatorFactory.getAndMakeChild(PlanUtils.getReduceSinkDesc(reduceKeys, - reduceValues, outputColumnNames, true, -1, numPartitionFields, - numReducers), new RowSchema(reduceSinkOutputRowResolver2 - .getColumnInfos()), groupByOperatorInfo), + reduceValues, outputColumnNames, true, -1, numPartitionFields, + numReducers), new RowSchema(reduceSinkOutputRowResolver2 + .getColumnInfos()), groupByOperatorInfo), reduceSinkOutputRowResolver2); rsOp.setColumnExprMap(colExprMap); @@ -3644,11 +3703,11 @@ // For grouping sets, add a dummy grouping key if (groupingSetsPresent) { addGroupingSetKey( - groupByKeys, - groupByInputRowResolver2, - groupByOutputRowResolver2, - outputColumnNames, - colExprMap); + groupByKeys, + groupByInputRowResolver2, + groupByOutputRowResolver2, + outputColumnNames, + colExprMap); } HashMap aggregationTrees = parseInfo @@ -3664,7 +3723,7 @@ assert (paraExpression != null); aggParameters.add(new ExprNodeColumnDesc(paraExprInfo.getType(), paraExpression, paraExprInfo.getTabAlias(), paraExprInfo - .getIsVirtualCol())); + .getIsVirtualCol())); String aggName = unescapeIdentifier(value.getChild(0).getText()); @@ -3682,7 +3741,7 @@ udaf.genericUDAFEvaluator, udaf.convertedParameters, (mode != GroupByDesc.Mode.FINAL && value.getToken().getType() == - HiveParser.TOK_FUNCTIONDI), + HiveParser.TOK_FUNCTIONDI), amode)); String field = getColumnInternalName(groupByKeys.size() + aggregations.size() - 1); @@ -3691,13 +3750,14 @@ field, udaf.returnType, "", false)); } float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); - float memoryThreshold = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + float memoryThreshold = HiveConf + .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( - new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, - false,groupByMemoryUsage,memoryThreshold, null, false, 0), - new RowSchema(groupByOutputRowResolver2.getColumnInfos()), - reduceSinkOperatorInfo2), groupByOutputRowResolver2); + new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, + false, groupByMemoryUsage, memoryThreshold, null, false, 0), + new RowSchema(groupByOutputRowResolver2.getColumnInfos()), + reduceSinkOperatorInfo2), groupByOutputRowResolver2); op.setColumnExprMap(colExprMap); return op; } @@ -3735,7 +3795,7 @@ int numReducers = -1; ObjectPair, List> grpByExprsGroupingSets = - getGroupByGroupingSetsForClause(parseInfo, dest); + getGroupByGroupingSetsForClause(parseInfo, dest); List grpByExprs = grpByExprsGroupingSets.getFirst(); List groupingSets = grpByExprsGroupingSets.getSecond(); @@ -3751,15 +3811,15 @@ // ////// 1. Generate ReduceSinkOperator Operator reduceSinkOperatorInfo = - genGroupByPlanReduceSinkOperator(qb, - dest, - input, - grpByExprs, - grpByExprs.size(), - false, - numReducers, - false, - false); + genGroupByPlanReduceSinkOperator(qb, + dest, + input, + grpByExprs, + grpByExprs.size(), + false, + numReducers, + false, + false); // ////// 2. Generate GroupbyOperator Operator groupByOperatorInfo = genGroupByPlanGroupByOperator(parseInfo, @@ -3787,7 +3847,7 @@ if (whereExpr != null) { OpParseContext inputCtx = opParseCtx.get(input); RowResolver inputRR = inputCtx.getRowResolver(); - ExprNodeDesc current = genExprNodeDesc((ASTNode)whereExpr.getChild(0), inputRR); + ExprNodeDesc current = genExprNodeDesc((ASTNode) whereExpr.getChild(0), inputRR); // Check the list of where expressions already added so they aren't duplicated ExprNodeDesc.ExprNodeDescEqualityWrapper currentWrapped = @@ -3824,8 +3884,8 @@ FilterDesc orFilterDesc = new FilterDesc(previous, false); selectInput = putOpInsertMap(OperatorFactory.getAndMakeChild( - orFilterDesc, new RowSchema( - inputRR.getColumnInfos()), input), inputRR); + orFilterDesc, new RowSchema( + inputRR.getColumnInfos()), input), inputRR); } // insert a select operator here used by the ColumnPruner to reduce @@ -3904,20 +3964,21 @@ // ////// Generate GroupbyOperator for a map-side partial aggregation Map genericUDAFEvaluators = - new LinkedHashMap(); + new LinkedHashMap(); QBParseInfo parseInfo = qb.getParseInfo(); // ////// 2. Generate GroupbyOperator Operator groupByOperatorInfo = genGroupByPlanGroupByOperator1(parseInfo, - dest, input, GroupByDesc.Mode.HASH, genericUDAFEvaluators, true, false); + dest, input, GroupByDesc.Mode.HASH, genericUDAFEvaluators, true, + null, false, false); int numReducers = -1; List grpByExprs = getGroupByForClause(parseInfo, dest); // ////// 3. Generate ReduceSinkOperator2 Operator reduceSinkOperatorInfo2 = genGroupByPlanReduceSinkOperator2MR( - parseInfo, dest, groupByOperatorInfo, grpByExprs.size(), numReducers, false); + parseInfo, dest, groupByOperatorInfo, grpByExprs.size(), numReducers, false); // ////// 4. Generate GroupbyOperator2 Operator groupByOperatorInfo2 = genGroupByPlanGroupByOperator2MR(parseInfo, @@ -3975,7 +4036,7 @@ QBParseInfo parseInfo = qb.getParseInfo(); ObjectPair, List> grpByExprsGroupingSets = - getGroupByGroupingSetsForClause(parseInfo, dest); + getGroupByGroupingSetsForClause(parseInfo, dest); List grpByExprs = grpByExprsGroupingSets.getFirst(); List groupingSets = grpByExprsGroupingSets.getSecond(); @@ -3995,22 +4056,22 @@ // operator. We set the numPartitionColumns to -1 for this purpose. This is // captured by WritableComparableHiveObject.hashCode() function. Operator reduceSinkOperatorInfo = - genGroupByPlanReduceSinkOperator(qb, - dest, - input, - grpByExprs, - (parseInfo.getDistinctFuncExprsForClause(dest).isEmpty() ? -1 : Integer.MAX_VALUE), - false, - -1, - false, - false); + genGroupByPlanReduceSinkOperator(qb, + dest, + input, + grpByExprs, + (parseInfo.getDistinctFuncExprsForClause(dest).isEmpty() ? -1 : Integer.MAX_VALUE), + false, + -1, + false, + false); // ////// 2. Generate GroupbyOperator Map genericUDAFEvaluators = - new LinkedHashMap(); + new LinkedHashMap(); GroupByOperator groupByOperatorInfo = (GroupByOperator) genGroupByPlanGroupByOperator( - parseInfo, dest, reduceSinkOperatorInfo, GroupByDesc.Mode.PARTIAL1, - genericUDAFEvaluators); + parseInfo, dest, reduceSinkOperatorInfo, GroupByDesc.Mode.PARTIAL1, + genericUDAFEvaluators); int numReducers = -1; if (grpByExprs.isEmpty()) { @@ -4019,12 +4080,12 @@ // ////// 3. Generate ReduceSinkOperator2 Operator reduceSinkOperatorInfo2 = genGroupByPlanReduceSinkOperator2MR( - parseInfo, dest, groupByOperatorInfo, grpByExprs.size(), numReducers, false); + parseInfo, dest, groupByOperatorInfo, grpByExprs.size(), numReducers, false); // ////// 4. Generate GroupbyOperator2 Operator groupByOperatorInfo2 = genGroupByPlanGroupByOperator2MR(parseInfo, - dest, reduceSinkOperatorInfo2, GroupByDesc.Mode.FINAL, - genericUDAFEvaluators, false); + dest, reduceSinkOperatorInfo2, GroupByDesc.Mode.FINAL, + genericUDAFEvaluators, false); return groupByOperatorInfo2; } @@ -4045,13 +4106,13 @@ static private void extractColumns(Set colNamesExprs, ExprNodeDesc exprNode) throws SemanticException { if (exprNode instanceof ExprNodeColumnDesc) { - colNamesExprs.add(((ExprNodeColumnDesc)exprNode).getColumn()); + colNamesExprs.add(((ExprNodeColumnDesc) exprNode).getColumn()); return; } if (exprNode instanceof ExprNodeGenericFuncDesc) { - ExprNodeGenericFuncDesc funcDesc = (ExprNodeGenericFuncDesc)exprNode; - for (ExprNodeDesc childExpr: funcDesc.getChildExprs()) { + ExprNodeGenericFuncDesc funcDesc = (ExprNodeGenericFuncDesc) exprNode; + for (ExprNodeDesc childExpr : funcDesc.getChildExprs()) { extractColumns(colNamesExprs, childExpr); } } @@ -4070,7 +4131,7 @@ private void checkExpressionsForGroupingSet(List grpByExprs, List distinctGrpByExprs, Map aggregationTrees, - RowResolver inputRowResolver) throws SemanticException { + RowResolver inputRowResolver) throws SemanticException { Set colNamesGroupByExprs = new HashSet(); Set colNamesGroupByDistinctExprs = new HashSet(); @@ -4084,7 +4145,7 @@ // If there is a distinctFuncExp, add all parameters to the reduceKeys. if (!distinctGrpByExprs.isEmpty()) { - for(ASTNode value: distinctGrpByExprs) { + for (ASTNode value : distinctGrpByExprs) { // 0 is function name for (int i = 1; i < value.getChildCount(); i++) { ASTNode parameter = (ASTNode) value.getChild(i); @@ -4094,8 +4155,7 @@ } if (hasCommonElement(colNamesGroupByExprs, colNamesGroupByDistinctExprs)) { - throw - new SemanticException(ErrorMsg.HIVE_GROUPING_SETS_AGGR_EXPRESSION_INVALID.getMsg()); + throw new SemanticException(ErrorMsg.HIVE_GROUPING_SETS_AGGR_EXPRESSION_INVALID.getMsg()); } } } @@ -4113,8 +4173,7 @@ } if (hasCommonElement(colNamesGroupByExprs, colNamesAggregateParameters)) { - throw - new SemanticException(ErrorMsg.HIVE_GROUPING_SETS_AGGR_EXPRESSION_INVALID.getMsg()); + throw new SemanticException(ErrorMsg.HIVE_GROUPING_SETS_AGGR_EXPRESSION_INVALID.getMsg()); } } } @@ -4126,52 +4185,92 @@ * spray by the group by key, and sort by the distinct key (if any), and * compute aggregates based on actual aggregates * - * The agggregation evaluation functions are as follows: Mapper: - * iterate/terminatePartial (mode = HASH) + * The agggregation evaluation functions are as follows: * + * No grouping sets: + * Group By Operator: + * grouping keys: group by expressions if no DISTINCT + * grouping keys: group by expressions + distinct keys if DISTINCT + * Mapper: iterate/terminatePartial (mode = HASH) * Partitioning Key: grouping key + * Sorting Key: grouping key if no DISTINCT + * grouping + distinct key if DISTINCT + * Reducer: iterate/terminate if DISTINCT + * merge/terminate if NO DISTINCT (mode MERGEPARTIAL) * - * Sorting Key: grouping key if no DISTINCT grouping + distinct key if - * DISTINCT + * Grouping Sets: + * Group By Operator: + * grouping keys: group by expressions + grouping id. if no DISTINCT + * grouping keys: group by expressions + grouping id. + distinct keys if DISTINCT + * Mapper: iterate/terminatePartial (mode = HASH) + * Partitioning Key: grouping key + grouping id. + * Sorting Key: grouping key + grouping id. if no DISTINCT + * grouping + grouping id. + distinct key if DISTINCT + * Reducer: iterate/terminate if DISTINCT + * merge/terminate if NO DISTINCT (mode MERGEPARTIAL) * - * Reducer: iterate/terminate if DISTINCT merge/terminate if NO DISTINCT (mode - * = MERGEPARTIAL) + * Grouping Sets with an additional MR job introduced (distincts are not allowed): + * Group By Operator: + * grouping keys: group by expressions + * Mapper: iterate/terminatePartial (mode = HASH) + * Partitioning Key: grouping key + * Sorting Key: grouping key + * Reducer: merge/terminate (mode MERGEPARTIAL) + * Group by Operator: + * grouping keys: group by expressions + add a new grouping id. key + * + * STAGE 2 + * Partitioning Key: grouping key + grouping id. + * Sorting Key: grouping key + grouping id. + * Reducer: merge/terminate (mode = FINAL) + * Group by Operator: + * grouping keys: group by expressions + grouping id. */ @SuppressWarnings("nls") - private Operator genGroupByPlanMapAggr1MR(String dest, QB qb, + private Operator genGroupByPlanMapAggrNoSkew(String dest, QB qb, Operator inputOperatorInfo) throws SemanticException { QBParseInfo parseInfo = qb.getParseInfo(); ObjectPair, List> grpByExprsGroupingSets = - getGroupByGroupingSetsForClause(parseInfo, dest); + getGroupByGroupingSetsForClause(parseInfo, dest); List grpByExprs = grpByExprsGroupingSets.getFirst(); List groupingSets = grpByExprsGroupingSets.getSecond(); boolean groupingSetsPresent = !groupingSets.isEmpty(); + int newMRJobGroupingSetsThreshold = + conf.getIntVar(HiveConf.ConfVars.HIVE_NEW_JOB_GROUPING_SET_CARDINALITY); + if (groupingSetsPresent) { checkExpressionsForGroupingSet(grpByExprs, - parseInfo.getDistinctFuncExprsForClause(dest), - parseInfo.getAggregationExprsForClause(dest), - opParseCtx.get(inputOperatorInfo).getRowResolver()); + parseInfo.getDistinctFuncExprsForClause(dest), + parseInfo.getAggregationExprsForClause(dest), + opParseCtx.get(inputOperatorInfo).getRowResolver()); } // ////// Generate GroupbyOperator for a map-side partial aggregation Map genericUDAFEvaluators = - new LinkedHashMap(); + new LinkedHashMap(); + + // Is the grouping sets data consumed in the current in MR job, or + // does it need an additional MR job + boolean groupingSetsNeedAdditionalMRJob = + groupingSetsPresent && groupingSets.size() > newMRJobGroupingSetsThreshold ? + true : false; + GroupByOperator groupByOperatorInfo = - (GroupByOperator) genGroupByPlanMapGroupByOperator( - qb, - dest, - grpByExprs, - inputOperatorInfo, - GroupByDesc.Mode.HASH, - genericUDAFEvaluators, - groupingSets, - groupingSetsPresent); + (GroupByOperator) genGroupByPlanMapGroupByOperator( + qb, + dest, + grpByExprs, + inputOperatorInfo, + GroupByDesc.Mode.HASH, + genericUDAFEvaluators, + groupingSets, + groupingSetsPresent && !groupingSetsNeedAdditionalMRJob); groupOpToInputTables.put(groupByOperatorInfo, opParseCtx.get( - inputOperatorInfo).getRowResolver().getTableNames()); + inputOperatorInfo).getRowResolver().getTableNames()); int numReducers = -1; // Optimize the scenario when there are no grouping keys - only 1 reducer is @@ -4181,27 +4280,64 @@ } // ////// Generate ReduceSink Operator + boolean isDistinct = !qb.getParseInfo().getDistinctFuncExprsForClause(dest).isEmpty(); + + // Distincts are not allowed with an additional mr job + if (groupingSetsNeedAdditionalMRJob && isDistinct) { + String errorMsg = "The number of rows per input row due to grouping sets is " + + groupingSets.size(); + throw new SemanticException( + ErrorMsg.HIVE_GROUPING_SETS_THRESHOLD_NOT_ALLOWED_WITH_DISTINCTS.getMsg(errorMsg)); + } + Operator reduceSinkOperatorInfo = - genGroupByPlanReduceSinkOperator(qb, - dest, - groupByOperatorInfo, - grpByExprs, - grpByExprs.size(), - true, - numReducers, - true, - groupingSetsPresent); + genGroupByPlanReduceSinkOperator(qb, + dest, + groupByOperatorInfo, + grpByExprs, + grpByExprs.size(), + true, + numReducers, + true, + groupingSetsPresent && !groupingSetsNeedAdditionalMRJob); - // This is a 1-stage map-reduce processing of the groupby. Tha map-side - // aggregates was just used to - // reduce output data. In case of distincts, partial results are not used, - // and so iterate is again - // invoked on the reducer. In case of non-distincts, partial results are - // used, and merge is invoked - // on the reducer. - return genGroupByPlanGroupByOperator1(parseInfo, dest, - reduceSinkOperatorInfo, GroupByDesc.Mode.MERGEPARTIAL, - genericUDAFEvaluators, false, groupingSetsPresent); + // Does it require a new MR job for grouping sets + if (!groupingSetsPresent || !groupingSetsNeedAdditionalMRJob) { + // This is a 1-stage map-reduce processing of the groupby. Tha map-side + // aggregates was just used to + // reduce output data. In case of distincts, partial results are not used, + // and so iterate is again + // invoked on the reducer. In case of non-distincts, partial results are + // used, and merge is invoked + // on the reducer. + return genGroupByPlanGroupByOperator1(parseInfo, dest, + reduceSinkOperatorInfo, GroupByDesc.Mode.MERGEPARTIAL, + genericUDAFEvaluators, false, + groupingSets, groupingSetsPresent, groupingSetsNeedAdditionalMRJob); + } + else + { + // Add 'n' rows corresponding to the grouping sets. For each row, create 'n' rows, + // one for each grouping set key. Since map-side aggregation has already been performed, + // the number of rows would have been reduced. Moreover, the rows corresponding to the + // grouping keys come together, so there is a higher chance of finding the rows in the hash + // table. + Operator groupByOperatorInfo2 = + genGroupByPlanGroupByOperator1(parseInfo, dest, + reduceSinkOperatorInfo, GroupByDesc.Mode.PARTIALS, + genericUDAFEvaluators, false, + groupingSets, groupingSetsPresent, groupingSetsNeedAdditionalMRJob); + + // ////// Generate ReduceSinkOperator2 + Operator reduceSinkOperatorInfo2 = genGroupByPlanReduceSinkOperator2MR( + parseInfo, dest, groupByOperatorInfo2, grpByExprs.size() + 1, numReducers, + groupingSetsPresent); + + // ////// Generate GroupbyOperator3 + return genGroupByPlanGroupByOperator2MR(parseInfo, dest, + reduceSinkOperatorInfo2, GroupByDesc.Mode.FINAL, + genericUDAFEvaluators, groupingSetsPresent); + } } /** @@ -4214,25 +4350,48 @@ * distinct is present) in hope of getting a uniform distribution, and compute * partial aggregates grouped by the reduction key (grouping key + distinct * key). Evaluate partial aggregates first, and spray by the grouping key to - * compute actual aggregates in the second phase. The agggregation evaluation - * functions are as follows: Mapper: iterate/terminatePartial (mode = HASH) + * compute actual aggregates in the second phase. * - * Partitioning Key: random() if no DISTINCT grouping + distinct key if - * DISTINCT + * The agggregation evaluation functions are as follows: * - * Sorting Key: grouping key if no DISTINCT grouping + distinct key if - * DISTINCT + * No grouping sets: + * STAGE 1 + * Group by Operator: + * grouping keys: group by expressions if no DISTINCT + * grouping keys: group by expressions + distinct keys if DISTINCT + * Mapper: iterate/terminatePartial (mode = HASH) + * Partitioning Key: random() if no DISTINCT + * grouping + distinct key if DISTINCT + * Sorting Key: grouping key if no DISTINCT + * grouping + distinct key if DISTINCT + * Reducer: iterate/terminatePartial if DISTINCT + * merge/terminatePartial if NO DISTINCT (mode = MERGEPARTIAL) + * Group by Operator: + * grouping keys: group by expressions * - * Reducer: iterate/terminatePartial if DISTINCT merge/terminatePartial if NO - * DISTINCT (mode = MERGEPARTIAL) - * * STAGE 2 + * Partitioning Key: grouping key + * Sorting Key: grouping key + * Reducer: merge/terminate (mode = FINAL) * - * Partitioining Key: grouping key + * In the presence of grouping sets, the agggregation evaluation functions are as follows: + * STAGE 1 + * Group by Operator: + * grouping keys: group by expressions + grouping id. if no DISTINCT + * grouping keys: group by expressions + + grouping id. + distinct keys if DISTINCT + * Mapper: iterate/terminatePartial (mode = HASH) + * Partitioning Key: random() if no DISTINCT + * grouping + grouping id. + distinct key if DISTINCT + * Sorting Key: grouping key + grouping id. if no DISTINCT + * grouping + grouping id. + distinct key if DISTINCT + * Reducer: iterate/terminatePartial if DISTINCT + * merge/terminatePartial if NO DISTINCT (mode = MERGEPARTIAL) + * Group by Operator: + * grouping keys: group by expressions + grouping id. * - * Sorting Key: grouping key if no DISTINCT grouping + distinct key if - * DISTINCT - * + * STAGE 2 + * Partitioning Key: grouping key + * Sorting Key: grouping key + grouping id. * Reducer: merge/terminate (mode = FINAL) */ @SuppressWarnings("nls") @@ -4242,7 +4401,7 @@ QBParseInfo parseInfo = qb.getParseInfo(); ObjectPair, List> grpByExprsGroupingSets = - getGroupByGroupingSetsForClause(parseInfo, dest); + getGroupByGroupingSetsForClause(parseInfo, dest); List grpByExprs = grpByExprsGroupingSets.getFirst(); List groupingSets = grpByExprsGroupingSets.getSecond(); @@ -4250,18 +4409,29 @@ if (groupingSetsPresent) { checkExpressionsForGroupingSet(grpByExprs, - parseInfo.getDistinctFuncExprsForClause(dest), - parseInfo.getAggregationExprsForClause(dest), - opParseCtx.get(inputOperatorInfo).getRowResolver()); + parseInfo.getDistinctFuncExprsForClause(dest), + parseInfo.getAggregationExprsForClause(dest), + opParseCtx.get(inputOperatorInfo).getRowResolver()); + + int newMRJobGroupingSetsThreshold = + conf.getIntVar(HiveConf.ConfVars.HIVE_NEW_JOB_GROUPING_SET_CARDINALITY); + + // Turn off skew if an additional MR job is required anyway for grouping sets. + if (groupingSets.size() > newMRJobGroupingSetsThreshold) { + String errorMsg = "The number of rows per input row due to grouping sets is " + + groupingSets.size(); + throw new SemanticException( + ErrorMsg.HIVE_GROUPING_SETS_THRESHOLD_NOT_ALLOWED_WITH_SKEW.getMsg(errorMsg)); + } } // ////// Generate GroupbyOperator for a map-side partial aggregation Map genericUDAFEvaluators = - new LinkedHashMap(); + new LinkedHashMap(); GroupByOperator groupByOperatorInfo = - (GroupByOperator) genGroupByPlanMapGroupByOperator( - qb, dest, grpByExprs, inputOperatorInfo, GroupByDesc.Mode.HASH, - genericUDAFEvaluators, groupingSets, groupingSetsPresent); + (GroupByOperator) genGroupByPlanMapGroupByOperator( + qb, dest, grpByExprs, inputOperatorInfo, GroupByDesc.Mode.HASH, + genericUDAFEvaluators, groupingSets, groupingSetsPresent); groupOpToInputTables.put(groupByOperatorInfo, opParseCtx.get( inputOperatorInfo).getRowResolver().getTableNames()); @@ -4273,20 +4443,21 @@ // ////// Generate ReduceSink Operator Operator reduceSinkOperatorInfo = - genGroupByPlanReduceSinkOperator(qb, - dest, - groupByOperatorInfo, - grpByExprs, - distinctFuncExprs.isEmpty() ? -1 : Integer.MAX_VALUE, - false, - -1, - true, - groupingSetsPresent); + genGroupByPlanReduceSinkOperator(qb, + dest, + groupByOperatorInfo, + grpByExprs, + distinctFuncExprs.isEmpty() ? -1 : Integer.MAX_VALUE, + false, + -1, + true, + groupingSetsPresent); // ////// Generate GroupbyOperator for a partial aggregation Operator groupByOperatorInfo2 = genGroupByPlanGroupByOperator1(parseInfo, - dest, reduceSinkOperatorInfo, GroupByDesc.Mode.PARTIALS, - genericUDAFEvaluators, false, groupingSetsPresent); + dest, reduceSinkOperatorInfo, GroupByDesc.Mode.PARTIALS, + genericUDAFEvaluators, false, + groupingSets, groupingSetsPresent, false); int numReducers = -1; if (grpByExprs.isEmpty()) { @@ -4295,31 +4466,31 @@ // ////// Generate ReduceSinkOperator2 Operator reduceSinkOperatorInfo2 = genGroupByPlanReduceSinkOperator2MR( - parseInfo, dest, groupByOperatorInfo2, grpByExprs.size(), numReducers, - groupingSetsPresent); + parseInfo, dest, groupByOperatorInfo2, grpByExprs.size(), numReducers, + groupingSetsPresent); // ////// Generate GroupbyOperator3 return genGroupByPlanGroupByOperator2MR(parseInfo, dest, - reduceSinkOperatorInfo2, GroupByDesc.Mode.FINAL, - genericUDAFEvaluators, groupingSetsPresent); + reduceSinkOperatorInfo2, GroupByDesc.Mode.FINAL, + genericUDAFEvaluators, groupingSetsPresent); } else { // If there are no grouping keys, grouping sets cannot be present assert !groupingSetsPresent; // ////// Generate ReduceSink Operator Operator reduceSinkOperatorInfo = - genGroupByPlanReduceSinkOperator(qb, - dest, - groupByOperatorInfo, - grpByExprs, - grpByExprs.size(), - false, - 1, - true, - groupingSetsPresent); + genGroupByPlanReduceSinkOperator(qb, + dest, + groupByOperatorInfo, + grpByExprs, + grpByExprs.size(), + false, + 1, + true, + groupingSetsPresent); return genGroupByPlanGroupByOperator2MR(parseInfo, dest, - reduceSinkOperatorInfo, GroupByDesc.Mode.FINAL, genericUDAFEvaluators, false); + reduceSinkOperatorInfo, GroupByDesc.Mode.FINAL, genericUDAFEvaluators, false); } } @@ -4346,10 +4517,10 @@ } private int getReducersBucketing(int totalFiles, int maxReducers) { - int numFiles = totalFiles/maxReducers; + int numFiles = totalFiles / maxReducers; while (true) { - if (totalFiles%numFiles == 0) { - return totalFiles/numFiles; + if (totalFiles % numFiles == 0) { + return totalFiles / numFiles; } numFiles++; } @@ -4358,8 +4529,8 @@ private static class SortBucketRSCtx { ArrayList partnCols; boolean multiFileSpray; - int numFiles; - int totalFiles; + int numFiles; + int totalFiles; public SortBucketRSCtx() { partnCols = null; @@ -4376,7 +4547,8 @@ } /** - * @param partnCols the partnCols to set + * @param partnCols + * the partnCols to set */ public void setPartnCols(ArrayList partnCols) { this.partnCols = partnCols; @@ -4390,7 +4562,8 @@ } /** - * @param multiFileSpray the multiFileSpray to set + * @param multiFileSpray + * the multiFileSpray to set */ public void setMultiFileSpray(boolean multiFileSpray) { this.multiFileSpray = multiFileSpray; @@ -4404,7 +4577,8 @@ } /** - * @param numFiles the numFiles to set + * @param numFiles + * the numFiles to set */ public void setNumFiles(int numFiles) { this.numFiles = numFiles; @@ -4418,7 +4592,8 @@ } /** - * @param totalFiles the totalFiles to set + * @param totalFiles + * the totalFiles to set */ public void setTotalFiles(int totalFiles) { this.totalFiles = totalFiles; @@ -4426,8 +4601,9 @@ } @SuppressWarnings("nls") - private Operator genBucketingSortingDest(String dest, Operator input, QB qb, TableDesc table_desc, - Table dest_tab, SortBucketRSCtx ctx) + private Operator genBucketingSortingDest(String dest, Operator input, QB qb, + TableDesc table_desc, + Table dest_tab, SortBucketRSCtx ctx) throws SemanticException { // If the table is bucketed, and bucketing is enforced, do the following: @@ -4437,20 +4613,21 @@ // spray the data into multiple buckets. That way, we can support a very large // number of buckets without needing a very large number of reducers. boolean enforceBucketing = false; - boolean enforceSorting = false; + boolean enforceSorting = false; ArrayList partnCols = new ArrayList(); ArrayList partnColsNoConvert = new ArrayList(); - ArrayList sortCols = new ArrayList(); + ArrayList sortCols = new ArrayList(); ArrayList sortOrders = new ArrayList(); boolean multiFileSpray = false; - int numFiles = 1; - int totalFiles = 1; + int numFiles = 1; + int totalFiles = 1; if ((dest_tab.getNumBuckets() > 0) && (conf.getBoolVar(HiveConf.ConfVars.HIVEENFORCEBUCKETING))) { enforceBucketing = true; partnCols = getParitionColsFromBucketCols(dest, qb, dest_tab, table_desc, input, true); - partnColsNoConvert = getParitionColsFromBucketCols(dest, qb, dest_tab, table_desc, input, false); + partnColsNoConvert = getParitionColsFromBucketCols(dest, qb, dest_tab, table_desc, input, + false); } if ((dest_tab.getSortCols() != null) && @@ -4470,7 +4647,7 @@ if (conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS) > 0) { maxReducers = conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS); } - int numBuckets = dest_tab.getNumBuckets(); + int numBuckets = dest_tab.getNumBuckets(); if (numBuckets > maxReducers) { multiFileSpray = true; totalFiles = numBuckets; @@ -4480,7 +4657,7 @@ else { // find the number of reducers such that it is a divisor of totalFiles maxReducers = getReducersBucketing(totalFiles, maxReducers); - numFiles = totalFiles/maxReducers; + numFiles = totalFiles / maxReducers; } } else { @@ -4499,6 +4676,7 @@ /** * Check for HOLD_DDLTIME hint. + * * @param qb * @return true if HOLD_DDLTIME is set, false otherwise. */ @@ -4524,7 +4702,7 @@ QBMetaData qbm = qb.getMetaData(); Integer dest_type = qbm.getDestTypeForAlias(dest); - Table dest_tab = null; // destination table if any + Table dest_tab = null; // destination table if any Partition dest_part = null;// destination partition if any String queryTmpdir = null; // the intermediate destination directory Path dest_path = null; // the final destination directory @@ -4545,7 +4723,7 @@ // Is the user trying to insert into a external tables if ((!conf.getBoolVar(HiveConf.ConfVars.HIVE_INSERT_INTO_EXTERNAL_TABLES)) && (dest_tab.getTableType().equals(TableType.EXTERNAL_TABLE))) { - throw new SemanticException( + throw new SemanticException( ErrorMsg.INSERT_EXTERNAL_TABLE.getMsg(dest_tab.getTableName())); } @@ -4555,17 +4733,17 @@ // check for partition List parts = dest_tab.getPartitionKeys(); if (parts != null && parts.size() > 0) { // table is partitioned - if (partSpec== null || partSpec.size() == 0) { // user did NOT specify partition + if (partSpec == null || partSpec.size() == 0) { // user did NOT specify partition throw new SemanticException(generateErrorMessage( - qb.getParseInfo().getDestForClause(dest), - ErrorMsg.NEED_PARTITION_ERROR.getMsg())); + qb.getParseInfo().getDestForClause(dest), + ErrorMsg.NEED_PARTITION_ERROR.getMsg())); } // the HOLD_DDLTIIME hint should not be used with dynamic partition since the // newly generated partitions should always update their DDLTIME if (holdDDLTime) { throw new SemanticException(generateErrorMessage( - qb.getParseInfo().getDestForClause(dest), - ErrorMsg.HOLD_DDLTIME_ON_NONEXIST_PARTITIONS.getMsg())); + qb.getParseInfo().getDestForClause(dest), + ErrorMsg.HOLD_DDLTIME_ON_NONEXIST_PARTITIONS.getMsg())); } dpCtx = qbm.getDPCtx(dest); if (dpCtx == null) { @@ -4579,8 +4757,8 @@ if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONING)) { // allow DP if (dpCtx.getNumDPCols() > 0 && (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEMERGEMAPFILES) || - HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEMERGEMAPREDFILES)) && - Utilities.supportCombineFileInputFormat() == false) { + HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEMERGEMAPREDFILES)) && + Utilities.supportCombineFileInputFormat() == false) { // Do not support merge for Hadoop versions (pre-0.20) that do not // support CombineHiveInputFormat HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVEMERGEMAPFILES, false); @@ -4591,8 +4769,8 @@ } else { // QBMetaData.DEST_PARTITION capture the all-SP case throw new SemanticException(generateErrorMessage( - qb.getParseInfo().getDestForClause(dest), - ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg())); + qb.getParseInfo().getDestForClause(dest), + ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg())); } if (dpCtx.getSPPath() != null) { dest_path = new Path(dest_tab.getPath(), dpCtx.getSPPath()); @@ -4662,7 +4840,7 @@ else { try { String ppath = dpCtx.getSPPath(); - ppath = ppath.substring(0, ppath.length()-1); + ppath = ppath.substring(0, ppath.length() - 1); DummyPartition p = new DummyPartition(dest_tab, dest_tab.getDbName() + "@" + dest_tab.getTableName() + "@" + ppath, @@ -4682,15 +4860,15 @@ dest_tab = dest_part.getTable(); if ((!conf.getBoolVar(HiveConf.ConfVars.HIVE_INSERT_INTO_EXTERNAL_TABLES)) && dest_tab.getTableType().equals(TableType.EXTERNAL_TABLE)) { - throw new SemanticException( + throw new SemanticException( ErrorMsg.INSERT_EXTERNAL_TABLE.getMsg(dest_tab.getTableName())); } Path tabPath = dest_tab.getPath(); Path partPath = dest_part.getPartitionPath(); - // if the table is in a different dfs than the partition, - // replace the partition's dfs with the table's dfs. + // if the table is in a different dfs than the partition, + // replace the partition's dfs with the table's dfs. dest_path = new Path(tabPath.toUri().getScheme(), tabPath.toUri() .getAuthority(), partPath.toUri().getPath()); @@ -4718,8 +4896,8 @@ Partition part = db.getPartition(dest_tab, dest_part.getSpec(), false); if (part == null) { throw new SemanticException(generateErrorMessage( - qb.getParseInfo().getDestForClause(dest), - ErrorMsg.HOLD_DDLTIME_ON_NONEXIST_PARTITIONS.getMsg())); + qb.getParseInfo().getDestForClause(dest), + ErrorMsg.HOLD_DDLTIME_ON_NONEXIST_PARTITIONS.getMsg())); } } catch (HiveException e) { throw new SemanticException(e); @@ -4919,16 +5097,16 @@ } Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(fileSinkDesc, - fsRS, input), inputRR); + fsRS, input), inputRR); if (ltd != null && SessionState.get() != null) { SessionState.get().getLineageState() - .mapDirToFop(ltd.getSourceDir(), (FileSinkOperator)output); + .mapDirToFop(ltd.getSourceDir(), (FileSinkOperator) output); } if (LOG.isDebugEnabled()) { LOG.debug("Created FileSink Plan for clause: " + dest + "dest_path: " - + dest_path + " row schema: " + inputRR.toString()); + + dest_path + " row schema: " + inputRR.toString()); } return output; @@ -4958,7 +5136,7 @@ int inColumnCnt = rowFields.size(); int outColumnCnt = tableFields.size(); if (dynPart && dpCtx != null) { - outColumnCnt += dpCtx.getNumDPCols(); + outColumnCnt += dpCtx.getNumDPCols(); } if (inColumnCnt != outColumnCnt) { @@ -4966,7 +5144,7 @@ + " columns, but query has " + inColumnCnt + " columns."; throw new SemanticException(ErrorMsg.TARGET_TABLE_COLUMN_MISMATCH.getMsg( qb.getParseInfo().getDestForClause(dest), reason)); - } else if (dynPart && dpCtx != null){ + } else if (dynPart && dpCtx != null) { // create the mapping from input ExprNode to dest table DP column dpCtx.mapInputToDP(rowFields.subList(tableFields.size(), rowFields.size())); } @@ -4997,8 +5175,8 @@ // JSON-format. if (!tableFieldTypeInfo.equals(rowFieldTypeInfo) && !(isLazySimpleSerDe - && tableFieldTypeInfo.getCategory().equals(Category.PRIMITIVE) && tableFieldTypeInfo - .equals(TypeInfoFactory.stringTypeInfo))) { + && tableFieldTypeInfo.getCategory().equals(Category.PRIMITIVE) && tableFieldTypeInfo + .equals(TypeInfoFactory.stringTypeInfo))) { // need to do some conversions here converted = true; if (tableFieldTypeInfo.getCategory() != Category.PRIMITIVE) { @@ -5023,7 +5201,7 @@ // deal with dynamic partition columns: convert ExprNodeDesc type to String?? if (dynPart && dpCtx != null && dpCtx.getNumDPCols() > 0) { // DP columns starts with tableFields.size() - for (int i = tableFields.size(); i < rowFields.size(); ++i ) { + for (int i = tableFields.size(); i < rowFields.size(); ++i) { TypeInfo rowFieldTypeInfo = rowFields.get(i).getType(); ExprNodeDesc column = new ExprNodeColumnDesc( rowFieldTypeInfo, rowFields.get(i).getInternalName(), "", false); @@ -5044,7 +5222,7 @@ } Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new SelectDesc(expressions, colName), new RowSchema(rowResolver - .getColumnInfos()), input), rowResolver); + .getColumnInfos()), input), rowResolver); return output; } else { @@ -5074,7 +5252,7 @@ if (LOG.isDebugEnabled()) { LOG.debug("Created LimitOperator Plan for clause: " + dest - + " row schema: " + inputRR.toString()); + + " row schema: " + inputRR.toString()); } return limitMap; @@ -5104,7 +5282,7 @@ if (LOG.isDebugEnabled()) { LOG.debug("Table alias: " + outputTableAlias + " Col aliases: " - + colAliases); + + colAliases); } // Use the RowResolver from the input operator to generate a input @@ -5131,7 +5309,7 @@ if (numUdtfCols != numSuppliedAliases) { throw new SemanticException(ErrorMsg.UDTF_ALIAS_MISMATCH .getMsg("expected " + numUdtfCols + " aliases " + "but got " - + numSuppliedAliases)); + + numSuppliedAliases)); } // Generate the output column info's / row resolver using internal names. @@ -5187,10 +5365,10 @@ } private ArrayList getParitionColsFromBucketCols(String dest, QB qb, Table tab, - TableDesc table_desc, Operator input, boolean convert) - throws SemanticException { + TableDesc table_desc, Operator input, boolean convert) + throws SemanticException { List tabBucketCols = tab.getBucketCols(); - List tabCols = tab.getCols(); + List tabCols = tab.getCols(); // Partition by the bucketing column List posns = new ArrayList(); @@ -5209,8 +5387,9 @@ return genConvertCol(dest, qb, tab, table_desc, input, posns, convert); } - private ArrayList genConvertCol(String dest, QB qb, Table tab, TableDesc table_desc, Operator input, - List posns, boolean convert) throws SemanticException { + private ArrayList genConvertCol(String dest, QB qb, Table tab, + TableDesc table_desc, Operator input, + List posns, boolean convert) throws SemanticException { StructObjectInspector oi = null; try { Deserializer deserializer = table_desc.getDeserializerClass() @@ -5228,12 +5407,13 @@ // Check column type int columnNumber = posns.size(); ArrayList expressions = new ArrayList(columnNumber); - for (Integer posn: posns) { + for (Integer posn : posns) { ObjectInspector tableFieldOI = tableFields.get(posn).getFieldObjectInspector(); TypeInfo tableFieldTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(tableFieldOI); TypeInfo rowFieldTypeInfo = rowFields.get(posn).getType(); - ExprNodeDesc column = new ExprNodeColumnDesc(rowFieldTypeInfo, rowFields.get(posn).getInternalName(), - rowFields.get(posn).getTabAlias(), rowFields.get(posn).getIsVirtualCol()); + ExprNodeDesc column = new ExprNodeColumnDesc(rowFieldTypeInfo, rowFields.get(posn) + .getInternalName(), + rowFields.get(posn).getTabAlias(), rowFields.get(posn).getIsVirtualCol()); if (convert && !tableFieldTypeInfo.equals(rowFieldTypeInfo)) { // need to do some conversions here @@ -5242,14 +5422,14 @@ column = null; } else { column = TypeCheckProcFactory.DefaultExprProcessor - .getFuncExprNodeDesc(tableFieldTypeInfo.getTypeName(), - column); + .getFuncExprNodeDesc(tableFieldTypeInfo.getTypeName(), + column); } if (column == null) { String reason = "Cannot convert column " + posn + " from " - + rowFieldTypeInfo + " to " + tableFieldTypeInfo + "."; + + rowFieldTypeInfo + " to " + tableFieldTypeInfo + "."; throw new SemanticException(ErrorMsg.TARGET_TABLE_COLUMN_MISMATCH - .getMsg(qb.getParseInfo().getDestForClause(dest), reason)); + .getMsg(qb.getParseInfo().getDestForClause(dest), reason)); } } expressions.add(column); @@ -5258,11 +5438,12 @@ return expressions; } - private ArrayList getSortCols(String dest, QB qb, Table tab, TableDesc table_desc, Operator input, boolean convert) - throws SemanticException { + private ArrayList getSortCols(String dest, QB qb, Table tab, TableDesc table_desc, + Operator input, boolean convert) + throws SemanticException { RowResolver inputRR = opParseCtx.get(input).getRowResolver(); List tabSortCols = tab.getSortCols(); - List tabCols = tab.getCols(); + List tabCols = tab.getCols(); // Partition by the bucketing column List posns = new ArrayList(); @@ -5282,10 +5463,10 @@ } private ArrayList getSortOrders(String dest, QB qb, Table tab, Operator input) - throws SemanticException { + throws SemanticException { RowResolver inputRR = opParseCtx.get(input).getRowResolver(); List tabSortCols = tab.getSortCols(); - List tabCols = tab.getCols(); + List tabCols = tab.getCols(); ArrayList orders = new ArrayList(); for (Order sortCol : tabSortCols) { @@ -5301,11 +5482,11 @@ @SuppressWarnings("nls") private Operator genReduceSinkPlanForSortingBucketing(Table tab, Operator input, - ArrayList sortCols, - List sortOrders, - ArrayList partitionCols, - int numReducers) - throws SemanticException { + ArrayList sortCols, + List sortOrders, + ArrayList partitionCols, + int numReducers) + throws SemanticException { RowResolver inputRR = opParseCtx.get(input).getRowResolver(); // For the generation of the values expression just get the inputs @@ -5327,12 +5508,12 @@ StringBuilder order = new StringBuilder(); for (int sortOrder : sortOrders) { - order.append(sortOrder == BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC ? '+' :'-'); + order.append(sortOrder == BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC ? '+' : '-'); } Operator interim = putOpInsertMap(OperatorFactory.getAndMakeChild(PlanUtils .getReduceSinkDesc(sortCols, valueCols, outputColumns, false, -1, - partitionCols, order.toString(), numReducers), + partitionCols, order.toString(), numReducers), new RowSchema(inputRR.getColumnInfos()), input), inputRR); interim.setColumnExprMap(colExprMap); @@ -5350,12 +5531,12 @@ Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new ExtractDesc(new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, - Utilities.ReduceField.VALUE.toString(), "", false)), new RowSchema( - out_rwsch.getColumnInfos()), interim), out_rwsch); + Utilities.ReduceField.VALUE.toString(), "", false)), new RowSchema( + out_rwsch.getColumnInfos()), interim), out_rwsch); if (LOG.isDebugEnabled()) { LOG.debug("Created ReduceSink Plan for table: " + tab.getTableName() + - " row schema: " + out_rwsch.toString()); + " row schema: " + out_rwsch.toString()); } return output; @@ -5398,7 +5579,7 @@ "strict") && limit == null) { throw new SemanticException(generateErrorMessage(sortExprs, - ErrorMsg.NO_LIMIT_WITH_ORDERBY.getMsg())); + ErrorMsg.NO_LIMIT_WITH_ORDERBY.getMsg())); } } } @@ -5445,7 +5626,7 @@ } Operator interim = putOpInsertMap(OperatorFactory.getAndMakeChild(PlanUtils .getReduceSinkDesc(sortCols, valueCols, outputColumns, false, -1, - partitionCols, order.toString(), numReducers), + partitionCols, order.toString(), numReducers), new RowSchema(inputRR.getColumnInfos()), input), inputRR); interim.setColumnExprMap(colExprMap); @@ -5463,12 +5644,12 @@ Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new ExtractDesc(new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, - Utilities.ReduceField.VALUE.toString(), "", false)), new RowSchema( - out_rwsch.getColumnInfos()), interim), out_rwsch); + Utilities.ReduceField.VALUE.toString(), "", false)), new RowSchema( + out_rwsch.getColumnInfos()), interim), out_rwsch); if (LOG.isDebugEnabled()) { LOG.debug("Created ReduceSink Plan for clause: " + dest + " row schema: " - + out_rwsch.toString()); + + out_rwsch.toString()); } return output; } @@ -5487,7 +5668,7 @@ Map colExprMap = new HashMap(); HashMap> posToAliasMap = new HashMap>(); HashMap> filterMap = - new HashMap>(); + new HashMap>(); for (int pos = 0; pos < right.length; ++pos) { @@ -5628,9 +5809,9 @@ ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap( OperatorFactory.getAndMakeChild(PlanUtils.getReduceSinkDesc(reduceKeys, - reduceValues, outputColumns, false, joinTree.getNextTag(), - reduceKeys.size(), numReds), new RowSchema(outputRS - .getColumnInfos()), child), outputRS); + reduceValues, outputColumns, false, joinTree.getNextTag(), + reduceKeys.size(), numReds), new RowSchema(outputRS + .getColumnInfos()), child), outputRS); rsOp.setColumnExprMap(colExprMap); return rsOp; } @@ -5724,7 +5905,7 @@ // create selection operator Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new SelectDesc(colList, columnNames, false), new RowSchema(inputRR - .getColumnInfos()), input), inputRR); + .getColumnInfos()), input), inputRR); output.setColumnExprMap(input.getColumnExprMap()); return output; @@ -5773,12 +5954,13 @@ // Generate group-by operator float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); - float memoryThreshold = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + float memoryThreshold = HiveConf + .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( - new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, - false,groupByMemoryUsage,memoryThreshold, null, false, 0), - new RowSchema(groupByOutputRowResolver.getColumnInfos()), - inputOperatorInfo), groupByOutputRowResolver); + new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, + false, groupByMemoryUsage, memoryThreshold, null, false, 0), + new RowSchema(groupByOutputRowResolver.getColumnInfos()), + inputOperatorInfo), groupByOutputRowResolver); op.setColumnExprMap(colExprMap); return op; @@ -5810,7 +5992,7 @@ if (commonType == null) { throw new SemanticException( "Cannot do equality join on different types: " + a.getTypeName() - + " and " + b.getTypeName()); + + " and " + b.getTypeName()); } } // Add implicit type conversion if necessary @@ -5819,7 +6001,7 @@ keys.get(i).set( k, TypeCheckProcFactory.DefaultExprProcessor.getFuncExprNodeDesc( - commonType.getTypeName(), keys.get(i).get(k))); + commonType.getTypeName(), keys.get(i).get(k))); } } } @@ -5926,7 +6108,7 @@ String alias = child.getChildCount() == 1 ? tableName : unescapeIdentifier(child.getChild(child.getChildCount() - 1) - .getText().toLowerCase()); + .getText().toLowerCase()); if (i == 0) { leftAliases.add(alias); @@ -6031,10 +6213,10 @@ if ((left.getToken().getType() == HiveParser.TOK_TABREF) || (left.getToken().getType() == HiveParser.TOK_SUBQUERY)) { String tableName = getUnescapedUnqualifiedTableName((ASTNode) left.getChild(0)) - .toLowerCase(); + .toLowerCase(); String alias = left.getChildCount() == 1 ? tableName : unescapeIdentifier(left.getChild(left.getChildCount() - 1) - .getText().toLowerCase()); + .getText().toLowerCase()); joinTree.setLeftAlias(alias); String[] leftAliases = new String[1]; leftAliases[0] = alias; @@ -6062,10 +6244,10 @@ if ((right.getToken().getType() == HiveParser.TOK_TABREF) || (right.getToken().getType() == HiveParser.TOK_SUBQUERY)) { String tableName = getUnescapedUnqualifiedTableName((ASTNode) right.getChild(0)) - .toLowerCase(); + .toLowerCase(); String alias = right.getChildCount() == 1 ? tableName : unescapeIdentifier(right.getChild(right.getChildCount() - 1) - .getText().toLowerCase()); + .getText().toLowerCase()); String[] rightAliases = new String[1]; rightAliases[0] = alias; joinTree.setRightAliases(rightAliases); @@ -6102,7 +6284,7 @@ joinTree.setFilterMap(new int[2][]); ArrayList> filtersForPushing = - new ArrayList>(); + new ArrayList>(); filtersForPushing.add(new ArrayList()); filtersForPushing.add(new ArrayList()); joinTree.setFiltersForPushing(filtersForPushing); @@ -6205,7 +6387,7 @@ ArrayList nns = node.getNullSafes(); ArrayList tns = target.getNullSafes(); for (int i = 0; i < tns.size(); i++) { - tns.set(i, tns.get(i) & nns.get(i)); // any of condition contains non-NS, non-NS + tns.set(i, tns.get(i) & nns.get(i)); // any of condition contains non-NS, non-NS } ArrayList> filters = target.getFilters(); @@ -6224,7 +6406,7 @@ for (int[] mapping : nmap) { if (mapping != null) { - for (int i = 0; i < mapping.length; i+=2) { + for (int i = 0; i < mapping.length; i += 2) { if (pos > 0 || mapping[i] > 0) { mapping[i] += trgtRightAliases.length; } @@ -6410,7 +6592,7 @@ } Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new SelectDesc(colList, columnNames, true), new RowSchema(inputRR - .getColumnInfos()), input), inputRR); + .getColumnInfos()), input), inputRR); output.setColumnExprMap(columnExprMap); return output; } @@ -6462,7 +6644,7 @@ } List currASTList = new ArrayList(); - for (ASTNode value: list) { + for (ASTNode value : list) { // 0 is function name for (int i = 1; i < value.getChildCount(); i++) { ASTNode parameter = (ASTNode) value.getChild(i); @@ -6564,8 +6746,8 @@ ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap( OperatorFactory.getAndMakeChild(PlanUtils.getReduceSinkDesc(reduceKeys, - reduceValues, outputColumnNames, true, -1, reduceKeys.size(), -1), - new RowSchema(reduceSinkOutputRowResolver.getColumnInfos()), input), + reduceValues, outputColumnNames, true, -1, reduceKeys.size(), -1), + new RowSchema(reduceSinkOutputRowResolver.getColumnInfos()), input), reduceSinkOutputRowResolver); rsOp.setColumnExprMap(colExprMap); @@ -6573,7 +6755,7 @@ } // Groups the clause names into lists so that any two clauses in the same list has the same - // group by and distinct keys and no clause appears in more than one list. Returns a list of the + // group by and distinct keys and no clause appears in more than one list. Returns a list of the // lists of clauses. private List> getCommonGroupByDestGroups(QB qb, Operator input) throws SemanticException { @@ -6588,7 +6770,7 @@ // If this is a trivial query block return if (ks.size() <= 1) { - List oneList = new ArrayList(1); + List oneList = new ArrayList(1); if (ks.size() == 1) { oneList.add(ks.first()); } @@ -6607,7 +6789,7 @@ // Add the group by expressions List grpByExprs = getGroupByForClause(qbp, dest); - for (ASTNode grpByExpr: grpByExprs) { + for (ASTNode grpByExpr : grpByExprs) { ExprNodeDesc.ExprNodeDescEqualityWrapper grpByExprWrapper = new ExprNodeDesc.ExprNodeDescEqualityWrapper(genExprNodeDesc(grpByExpr, inputRR)); if (!sprayKeys.contains(grpByExprWrapper)) { @@ -6667,7 +6849,7 @@ List distinctExprs = new ArrayList(); - for (ASTNode distinctAggExpr: distinctAggExprs) { + for (ASTNode distinctAggExpr : distinctAggExprs) { // 0 is function name for (int i = 1; i < distinctAggExpr.getChildCount(); i++) { ASTNode parameter = (ASTNode) distinctAggExpr.getChild(i); @@ -6778,7 +6960,7 @@ // Constructs a standard group by plan if: // There is no other subquery with the same group by/distinct keys or // (There are no aggregations in a representative query for the group and - // There is no group by in that representative query) or + // There is no group by in that representative query) or // The data is skewed or // The conf variable used to control combining group bys into a signle reducer is false if (commonGroupByDestGroup.size() == 1 || @@ -6797,9 +6979,9 @@ if (qbp.getAggregationExprsForClause(dest).size() != 0 || getGroupByForClause(qbp, dest).size() > 0) { - //multiple distincts is not supported with skew in data + // multiple distincts is not supported with skew in data if (conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) && - qbp.getDistinctFuncExprsForClause(dest).size() > 1) { + qbp.getDistinctFuncExprsForClause(dest).size() > 1) { throw new SemanticException(ErrorMsg.UNSUPPORTED_MULTIPLE_DISTINCTS. getMsg()); } @@ -6808,7 +6990,7 @@ curr = insertSelectAllPlanForGroupBy(curr); if (conf.getBoolVar(HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE)) { if (!conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { - curr = genGroupByPlanMapAggr1MR(dest, qb, curr); + curr = genGroupByPlanMapAggrNoSkew(dest, qb, curr); } else { curr = genGroupByPlanMapAggr2MR(dest, qb, curr); } @@ -6943,7 +7125,7 @@ HashMap rightmap = rightRR.getFieldMap(rightalias); // make sure the schemas of both sides are the same ASTNode tabref = qb.getAliases().isEmpty() ? null : - qb.getParseInfo().getSrcForAlias(qb.getAliases().get(0)); + qb.getParseInfo().getSrcForAlias(qb.getAliases().get(0)); if (leftmap.size() != rightmap.size()) { throw new SemanticException("Schema of both sides of union should match."); } @@ -6954,31 +7136,31 @@ if (rInfo == null) { throw new SemanticException(generateErrorMessage(tabref, "Schema of both sides of union should match. " + rightalias - + " does not have the field " + field)); + + " does not have the field " + field)); } if (lInfo == null) { throw new SemanticException(generateErrorMessage(tabref, "Schema of both sides of union should match. " + leftalias - + " does not have the field " + field)); + + " does not have the field " + field)); } if (!lInfo.getInternalName().equals(rInfo.getInternalName())) { throw new SemanticException(generateErrorMessage(tabref, "Schema of both sides of union should match: field " + field + ":" - + " appears on the left side of the UNION at column position: " + - getPositionFromInternalName(lInfo.getInternalName()) - + ", and on the right side of the UNION at column position: " + - getPositionFromInternalName(rInfo.getInternalName()) - + ". Column positions should match for a UNION")); + + " appears on the left side of the UNION at column position: " + + getPositionFromInternalName(lInfo.getInternalName()) + + ", and on the right side of the UNION at column position: " + + getPositionFromInternalName(rInfo.getInternalName()) + + ". Column positions should match for a UNION")); } - //try widening coversion, otherwise fail union + // try widening coversion, otherwise fail union TypeInfo commonTypeInfo = FunctionRegistry.getCommonClassForUnionAll(lInfo.getType(), rInfo.getType()); if (commonTypeInfo == null) { throw new SemanticException(generateErrorMessage(tabref, "Schema of both sides of union should match: Column " + field - + " is of type " + lInfo.getType().getTypeName() - + " on first table and type " + rInfo.getType().getTypeName() - + " on second table")); + + " is of type " + lInfo.getType().getTypeName() + + " on first table and type " + rInfo.getType().getTypeName() + + " on second table")); } } @@ -6990,7 +7172,7 @@ ColumnInfo rInfo = rightmap.get(field); ColumnInfo unionColInfo = new ColumnInfo(lInfo); unionColInfo.setType(FunctionRegistry.getCommonClassForUnionAll(lInfo.getType(), - rInfo.getType())); + rInfo.getType())); unionoutRR.put(unionalias, field, unionColInfo); } @@ -7008,7 +7190,7 @@ if (leftOp instanceof UnionOperator) { // make left a child of right List> child = - new ArrayList>(); + new ArrayList>(); child.add(leftOp); rightOp.setChildOperators(child); @@ -7022,7 +7204,7 @@ } else { // make right a child of left List> child = - new ArrayList>(); + new ArrayList>(); child.add(rightOp); leftOp.setChildOperators(child); @@ -7039,11 +7221,11 @@ // Create a new union operator Operator unionforward = OperatorFactory .getAndMakeChild(new UnionDesc(), new RowSchema(unionoutRR - .getColumnInfos())); + .getColumnInfos())); // set union operator as child of each of leftOp and rightOp List> child = - new ArrayList>(); + new ArrayList>(); child.add(unionforward); rightOp.setChildOperators(child); @@ -7052,7 +7234,7 @@ leftOp.setChildOperators(child); List> parent = - new ArrayList>(); + new ArrayList>(); parent.add(leftOp); parent.add(rightOp); unionforward.setParentOperators(parent); @@ -7063,9 +7245,9 @@ /** * Generates a select operator which can go between the original input operator and the union - * operator. This select casts columns to match the type of the associated column in the union, - * other columns pass through unchanged. The new operator's only parent is the original input - * operator to the union, and it's only child is the union. If the input does not need to be + * operator. This select casts columns to match the type of the associated column in the union, + * other columns pass through unchanged. The new operator's only parent is the original input + * operator to the union, and it's only child is the union. If the input does not need to be * cast, the original operator is returned, and no new select operator is added. * * @param origInputOp @@ -7084,15 +7266,16 @@ private Operator genInputSelectForUnion( Operator origInputOp, Map origInputFieldMap, String origInputAlias, RowResolver unionoutRR, String unionalias) - throws UDFArgumentException { + throws UDFArgumentException { List columns = new ArrayList(); boolean needsCast = false; - for (Map.Entry unionEntry: unionoutRR.getFieldMap(unionalias).entrySet()) { + for (Map.Entry unionEntry : unionoutRR.getFieldMap(unionalias).entrySet()) { String field = unionEntry.getKey(); ColumnInfo lInfo = origInputFieldMap.get(field); ExprNodeDesc column = new ExprNodeColumnDesc(lInfo.getType(), lInfo.getInternalName(), - lInfo.getTabAlias(), lInfo.getIsVirtualCol(), lInfo.isSkewedCol());; + lInfo.getTabAlias(), lInfo.getIsVirtualCol(), lInfo.isSkewedCol()); + ; if (!lInfo.getType().equals(unionEntry.getValue().getType())) { needsCast = true; column = TypeCheckProcFactory.DefaultExprProcessor.getFuncExprNodeDesc( @@ -7236,10 +7419,10 @@ * if the column is a skewed column, use ColumnInfo accordingly */ ColumnInfo colInfo = new ColumnInfo(fields.get(i).getFieldName(), - TypeInfoUtils.getTypeInfoFromObjectInspector(fields.get(i) - .getFieldObjectInspector()), alias, false); + TypeInfoUtils.getTypeInfoFromObjectInspector(fields.get(i) + .getFieldObjectInspector()), alias, false); colInfo.setSkewedCol((isSkewedCol(alias, qb, fields.get(i) - .getFieldName())) ? true : false); + .getFieldName())) ? true : false); rwsch.put(alias, fields.get(i).getFieldName(), colInfo); } } catch (SerDeException e) { @@ -7255,9 +7438,9 @@ TypeInfoFactory.stringTypeInfo, alias, true)); } - //put all virutal columns in RowResolver. + // put all virutal columns in RowResolver. Iterator vcs = VirtualColumn.getRegistry(conf).iterator(); - //use a list for easy cumtomize + // use a list for easy cumtomize List vcList = new ArrayList(); while (vcs.hasNext()) { VirtualColumn vc = vcs.next(); @@ -7277,7 +7460,7 @@ } top = putOpInsertMap(OperatorFactory.get(tsDesc, - new RowSchema(rwsch.getColumnInfos())), rwsch); + new RowSchema(rwsch.getColumnInfos())), rwsch); // Add this to the list of top operators - we always start from a table // scan @@ -7311,7 +7494,7 @@ if (num > den) { throw new SemanticException( ErrorMsg.BUCKETED_NUMERATOR_BIGGER_DENOMINATOR.getMsg() + " " - + tab.getTableName()); + + tab.getTableName()); } // check if a predicate is needed @@ -7354,7 +7537,7 @@ colsEqual, alias, rwsch, qb.getMetaData(), null); tableOp = OperatorFactory.getAndMakeChild(new FilterDesc( samplePredicate, true, new sampleDesc(ts.getNumerator(), ts - .getDenominator(), tabBucketCols, true)), + .getDenominator(), tabBucketCols, true)), new RowSchema(rwsch.getColumnInfos()), top); } else { // need to add filter @@ -7394,9 +7577,9 @@ .getBucketCols(), true, alias, rwsch, qb.getMetaData(), null); tableOp = OperatorFactory .getAndMakeChild(new FilterDesc(samplePred, true, - new sampleDesc(tsSample.getNumerator(), tsSample - .getDenominator(), tab.getBucketCols(), true)), - new RowSchema(rwsch.getColumnInfos()), top); + new sampleDesc(tsSample.getNumerator(), tsSample + .getDenominator(), tab.getBucketCols(), true)), + new RowSchema(rwsch.getColumnInfos()), top); LOG.info("No need for sample filter"); } else { // The table is not bucketed, add a dummy filter :: rand() @@ -7407,7 +7590,7 @@ LOG.info("Need sample filter"); ExprNodeDesc randFunc = TypeCheckProcFactory.DefaultExprProcessor .getFuncExprNodeDesc("rand", new ExprNodeConstantDesc(Integer - .valueOf(460476415))); + .valueOf(460476415))); ExprNodeDesc samplePred = genSamplePredicate(tsSample, null, false, alias, rwsch, qb.getMetaData(), randFunc); tableOp = OperatorFactory.getAndMakeChild(new FilterDesc( @@ -7430,7 +7613,7 @@ private boolean isSkewedCol(String alias, QB qb, String colName) { boolean isSkewedCol = false; List skewedCols = qb.getSkewedColumnNames(alias); - for (String skewedCol:skewedCols) { + for (String skewedCol : skewedCols) { if (skewedCol.equalsIgnoreCase(colName)) { isSkewedCol = true; } @@ -7438,7 +7621,8 @@ return isSkewedCol; } - private void setupStats(TableScanDesc tsDesc, QBParseInfo qbp, Table tab, String alias, RowResolver rwsch) + private void setupStats(TableScanDesc tsDesc, QBParseInfo qbp, Table tab, String alias, + RowResolver rwsch) throws SemanticException { if (!qbp.isAnalyzeCommand()) { @@ -7473,7 +7657,7 @@ // Theoretically the key prefix could be any unique string shared // between TableScanOperator (when publishing) and StatsTask (when aggregating). // Here we use - // table_name + partitionSec + // table_name + partitionSec // as the prefix for easy of read during explain and debugging. // Currently, partition spec can only be static partition. String k = tblName + Path.SEPARATOR; @@ -7604,7 +7788,7 @@ RowResolver lvForwardRR = new RowResolver(); RowResolver source = opParseCtx.get(op).getRowResolver(); for (ColumnInfo col : source.getColumnInfos()) { - if(col.getIsVirtualCol() && col.isHiddenVirtualCol()) { + if (col.getIsVirtualCol() && col.isHiddenVirtualCol()) { continue; } String[] tabCol = source.reverseLookup(col.getInternalName()); @@ -7621,10 +7805,10 @@ // Get the all path by making a select(*). RowResolver allPathRR = opParseCtx.get(lvForward).getRowResolver(); - //Operator allPath = op; + // Operator allPath = op; Operator allPath = putOpInsertMap(OperatorFactory.getAndMakeChild( - new SelectDesc(true), new RowSchema(allPathRR.getColumnInfos()), - lvForward), allPathRR); + new SelectDesc(true), new RowSchema(allPathRR.getColumnInfos()), + lvForward), allPathRR); // Get the UDTF Path QB blankQb = new QB(null, null, false); Operator udtfPath = genSelectPlan((ASTNode) lateralViewTree @@ -7652,7 +7836,7 @@ // LVmerge.. in the above order Map colExprMap = new HashMap(); - int i=0; + int i = 0; for (ColumnInfo c : allPathRR.getColumnInfos()) { String internalName = getColumnInternalName(i); i++; @@ -7705,7 +7889,7 @@ /** * A helper function to generate a column stats task on top of map-red task. The column stats * task fetches from the output of the map-red task, constructs the column stats object and - * persists it to the metastore. + * persists it to the metastore. * * This method generates a plan with a column stats task on top of map-red task and sets up the * appropriate metadata to be used during execution. @@ -7733,7 +7917,7 @@ resultTab, qb.getParseInfo().getOuterQueryLimit()); ColumnStatsDesc cStatsDesc = new ColumnStatsDesc(tableName, partName, - colName, colType, isTblLevel); + colName, colType, isTblLevel); cStatsWork = new ColumnStatsWork(fetch, cStatsDesc); cStatsTask = (ColumnStatsTask) TaskFactory.get(cStatsWork, conf); rootTasks.add(cStatsTask); @@ -7752,7 +7936,8 @@ initParseCtx(pCtx); List> mvTask = new ArrayList>(); - /* In case of a select, use a fetch task instead of a move task. + /* + * In case of a select, use a fetch task instead of a move task. * If the select is from analyze table column rewrite, don't create a fetch task. Instead create * a column stats task later. */ @@ -7790,12 +7975,14 @@ if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEINDEXAUTOUPDATE)) { IndexUpdater indexUpdater = new IndexUpdater(loadTableWork, getInputs(), conf); try { - List> indexUpdateTasks = indexUpdater.generateUpdateTasks(); + List> indexUpdateTasks = indexUpdater + .generateUpdateTasks(); for (Task updateTask : indexUpdateTasks) { tsk.addDependentTask(updateTask); } } catch (HiveException e) { - console.printInfo("WARNING: could not auto-update stale indexes, indexes are not in of sync"); + console + .printInfo("WARNING: could not auto-update stale indexes, which are not in sync"); } } } @@ -7852,41 +8039,41 @@ // The dispatcher generates the plan from the operator tree Map opRules = new LinkedHashMap(); opRules.put(new RuleRegExp(new String("R1"), - TableScanOperator.getOperatorName() + "%"), - new GenMRTableScan1()); + TableScanOperator.getOperatorName() + "%"), + new GenMRTableScan1()); opRules.put(new RuleRegExp(new String("R2"), - TableScanOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"), - new GenMRRedSink1()); + TableScanOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"), + new GenMRRedSink1()); opRules.put(new RuleRegExp(new String("R3"), - ReduceSinkOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"), - new GenMRRedSink2()); + ReduceSinkOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"), + new GenMRRedSink2()); opRules.put(new RuleRegExp(new String("R4"), - FileSinkOperator.getOperatorName() + "%"), - new GenMRFileSink1()); + FileSinkOperator.getOperatorName() + "%"), + new GenMRFileSink1()); opRules.put(new RuleRegExp(new String("R5"), - UnionOperator.getOperatorName() + "%"), - new GenMRUnion1()); + UnionOperator.getOperatorName() + "%"), + new GenMRUnion1()); opRules.put(new RuleRegExp(new String("R6"), - UnionOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"), - new GenMRRedSink3()); + UnionOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"), + new GenMRRedSink3()); opRules.put(new RuleRegExp(new String("R6"), - MapJoinOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"), - new GenMRRedSink4()); + MapJoinOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"), + new GenMRRedSink4()); opRules.put(new RuleRegExp(new String("R7"), - TableScanOperator.getOperatorName() + "%.*" + MapJoinOperator.getOperatorName() + "%"), - MapJoinFactory.getTableScanMapJoin()); + TableScanOperator.getOperatorName() + "%.*" + MapJoinOperator.getOperatorName() + "%"), + MapJoinFactory.getTableScanMapJoin()); opRules.put(new RuleRegExp(new String("R8"), - ReduceSinkOperator.getOperatorName() + "%.*" + MapJoinOperator.getOperatorName() + "%"), - MapJoinFactory.getReduceSinkMapJoin()); + ReduceSinkOperator.getOperatorName() + "%.*" + MapJoinOperator.getOperatorName() + "%"), + MapJoinFactory.getReduceSinkMapJoin()); opRules.put(new RuleRegExp(new String("R9"), - UnionOperator.getOperatorName() + "%.*" + MapJoinOperator.getOperatorName() + "%"), - MapJoinFactory.getUnionMapJoin()); + UnionOperator.getOperatorName() + "%.*" + MapJoinOperator.getOperatorName() + "%"), + MapJoinFactory.getUnionMapJoin()); opRules.put(new RuleRegExp(new String("R10"), - MapJoinOperator.getOperatorName() + "%.*" + MapJoinOperator.getOperatorName() + "%"), - MapJoinFactory.getMapJoinMapJoin()); + MapJoinOperator.getOperatorName() + "%.*" + MapJoinOperator.getOperatorName() + "%"), + MapJoinFactory.getMapJoinMapJoin()); opRules.put(new RuleRegExp(new String("R11"), - MapJoinOperator.getOperatorName() + "%" + SelectOperator.getOperatorName() + "%"), - MapJoinFactory.getMapJoin()); + MapJoinOperator.getOperatorName() + "%" + SelectOperator.getOperatorName() + "%"), + MapJoinFactory.getMapJoin()); // The dispatcher fires the processor corresponding to the closest matching // rule and passes the context along @@ -7898,7 +8085,8 @@ topNodes.addAll(topOps.values()); ogw.startWalking(topNodes, null); - /* If the query was the result of analyze table column compute statistics rewrite, create + /* + * If the query was the result of analyze table column compute statistics rewrite, create * a column stats task instead of a fetch task to persist stats to the metastore. */ if (isCStats) { @@ -7959,12 +8147,12 @@ getLeafTasks(rootTasks, leaves); assert (leaves.size() > 0); for (Task task : leaves) { - if (task instanceof StatsTask){ - //StatsTask require table to already exist - for (Task parentOfStatsTask : task.getParentTasks()){ + if (task instanceof StatsTask) { + // StatsTask require table to already exist + for (Task parentOfStatsTask : task.getParentTasks()) { parentOfStatsTask.addDependentTask(crtTblTask); } - for (Task parentOfCrtTblTask : crtTblTask.getParentTasks()){ + for (Task parentOfCrtTblTask : crtTblTask.getParentTasks()) { parentOfCrtTblTask.removeDependentTask(task); } crtTblTask.addDependentTask(task); @@ -7976,8 +8164,8 @@ if (globalLimitCtx.isEnable() && fetchTask != null) { int fetchLimit = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVELIMITOPTMAXFETCH); - LOG.info("set least row check for FetchTask: " + globalLimitCtx.getGlobalLimit()); - fetchTask.getWork().setLeastNumRows(globalLimitCtx.getGlobalLimit()); + LOG.info("set least row check for FetchTask: " + globalLimitCtx.getGlobalLimit()); + fetchTask.getWork().setLeastNumRows(globalLimitCtx.getGlobalLimit()); } if (globalLimitCtx.isEnable() && globalLimitCtx.getLastReduceLimitDesc() != null) { @@ -8219,7 +8407,7 @@ } viewSelect = child; // prevent view from referencing itself - viewsExpanded.add(db.getCurrentDatabase()+"."+createVwDesc.getViewName()); + viewsExpanded.add(db.getCurrentDatabase() + "." + createVwDesc.getViewName()); } // continue analyzing from the child ASTNode. @@ -8234,7 +8422,7 @@ LOG.info("Completed getting MetaData in Semantic Analysis"); // Save the result schema derived from the sink operator produced - // by genPlan. This has the correct column names, which clients + // by genPlan. This has the correct column names, which clients // such as JDBC would prefer instead of the c0, c1 we'll end // up with later. Operator sinkOp = genPlan(qb); @@ -8299,8 +8487,8 @@ int derivedColCount = derivedSchema.size(); if (explicitColCount != derivedColCount) { throw new SemanticException(generateErrorMessage( - viewSelect, - ErrorMsg.VIEW_COL_MISMATCH.getMsg())); + viewSelect, + ErrorMsg.VIEW_COL_MISMATCH.getMsg())); } } @@ -8348,19 +8536,19 @@ if (createVwDesc.getPartColNames() != null) { // Make sure all partitioning columns referenced actually // exist and are in the correct order at the end - // of the list of columns produced by the view. Also move the field + // of the list of columns produced by the view. Also move the field // schema descriptors from derivedSchema to the partitioning key // descriptor. List partColNames = createVwDesc.getPartColNames(); if (partColNames.size() > derivedSchema.size()) { - throw new SemanticException( + throw new SemanticException( ErrorMsg.VIEW_PARTITION_MISMATCH.getMsg()); } // Get the partition columns from the end of derivedSchema. List partitionColumns = derivedSchema.subList( - derivedSchema.size() - partColNames.size(), - derivedSchema.size()); + derivedSchema.size() - partColNames.size(), + derivedSchema.size()); // Verify that the names match the PARTITIONED ON clause. Iterator colNameIter = partColNames.iterator(); @@ -8370,20 +8558,20 @@ FieldSchema fieldSchema = schemaIter.next(); if (!fieldSchema.getName().equals(colName)) { throw new SemanticException( - ErrorMsg.VIEW_PARTITION_MISMATCH.getMsg()); + ErrorMsg.VIEW_PARTITION_MISMATCH.getMsg()); } } - // Boundary case: require at least one non-partitioned column + // Boundary case: require at least one non-partitioned column // for consistency with tables. if (partColNames.size() == derivedSchema.size()) { - throw new SemanticException( + throw new SemanticException( ErrorMsg.VIEW_PARTITION_TOTAL.getMsg()); } // Now make a copy. createVwDesc.setPartCols( - new ArrayList(partitionColumns)); + new ArrayList(partitionColumns)); // Finally, remove the partition columns from the end of derivedSchema. // (Clearing the subList writes through to the underlying @@ -8444,7 +8632,7 @@ */ @SuppressWarnings("nls") public ExprNodeDesc genExprNodeDesc(ASTNode expr, RowResolver input, - TypeCheckCtx tcCtx) throws SemanticException { + TypeCheckCtx tcCtx) throws SemanticException { // We recursively create the exprNodeDesc. Base cases: when we encounter // a column ref, we convert that into an exprNodeColumnDesc; when we // encounter @@ -8464,15 +8652,15 @@ .getIsVirtualCol(), colInfo.isSkewedCol()); } - // Create the walker and the rules dispatcher. + // Create the walker and the rules dispatcher. tcCtx.setUnparseTranslator(unparseTranslator); HashMap nodeOutputs = - TypeCheckProcFactory.genExprNode(expr, tcCtx); + TypeCheckProcFactory.genExprNode(expr, tcCtx); ExprNodeDesc desc = (ExprNodeDesc) nodeOutputs.get(expr); if (desc == null) { String errMsg = tcCtx.getError(); - if ( errMsg == null) { + if (errMsg == null) { errMsg = "Error in parsing "; } throw new SemanticException(errMsg); @@ -8513,7 +8701,7 @@ public void validate() throws SemanticException { LOG.debug("validation start"); // Validate inputs and outputs have right protectmode to execute the query - for (ReadEntity readEntity: getInputs()) { + for (ReadEntity readEntity : getInputs()) { ReadEntity.Type type = readEntity.getType(); if (type != ReadEntity.Type.TABLE && @@ -8530,22 +8718,22 @@ if (tbl.isOffline()) { throw new SemanticException( ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg( - "Table " + tbl.getTableName())); + "Table " + tbl.getTableName())); } if (type == ReadEntity.Type.PARTITION && p != null && p.isOffline()) { throw new SemanticException( ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg( - "Table " + tbl.getTableName() + - " Partition " + p.getName())); + "Table " + tbl.getTableName() + + " Partition " + p.getName())); } } - for (WriteEntity writeEntity: getOutputs()) { + for (WriteEntity writeEntity : getOutputs()) { WriteEntity.Type type = writeEntity.getType(); - if(type == WriteEntity.Type.PARTITION || type == WriteEntity.Type.DUMMYPARTITION) { + if (type == WriteEntity.Type.PARTITION || type == WriteEntity.Type.DUMMYPARTITION) { String conflictingArchive; try { Partition usedp = writeEntity.getPartition(); @@ -8558,7 +8746,7 @@ } catch (HiveException e) { throw new SemanticException(e); } - if(conflictingArchive != null) { + if (conflictingArchive != null) { String message = String.format("Insert conflict with existing archive: %s", conflictingArchive); throw new SemanticException(message); @@ -8593,11 +8781,11 @@ throw new SemanticException(e); } - if (type == WriteEntity.Type.PARTITION && p!=null && p.isOffline()) { + if (type == WriteEntity.Type.PARTITION && p != null && p.isOffline()) { throw new SemanticException( ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg( - " Table " + tbl.getTableName() + - " Partition " + p.getName())); + " Table " + tbl.getTableName() + + " Partition " + p.getName())); } } @@ -8609,11 +8797,12 @@ if (tbl.isOffline()) { throw new SemanticException( ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg( - "Table " + tbl.getTableName())); + "Table " + tbl.getTableName())); } } - boolean reworkMapredWork = HiveConf.getBoolVar(this.conf, HiveConf.ConfVars.HIVE_REWORK_MAPREDWORK); + boolean reworkMapredWork = HiveConf.getBoolVar(this.conf, + HiveConf.ConfVars.HIVE_REWORK_MAPREDWORK); // validate all tasks for (Task rootTask : rootTasks) { @@ -8643,7 +8832,9 @@ /** * Add default properties for table property. If a default parameter exists * in the tblProp, the value in tblProp will be kept. - * @param table property map + * + * @param table + * property map * @return Modified table property map */ private Map addDefaultProperties(Map tblProp) { @@ -8655,7 +8846,7 @@ } String paraString = HiveConf.getVar(conf, ConfVars.NEWTABLEDEFAULTPARA); if (paraString != null && !paraString.isEmpty()) { - for (String keyValuePair: paraString.split(",")) { + for (String keyValuePair : paraString.split(",")) { String[] keyValue = keyValuePair.split("=", 2); if (keyValue.length != 2) { continue; @@ -8678,7 +8869,7 @@ */ private ASTNode analyzeCreateTable(ASTNode ast, QB qb) throws SemanticException { - String tableName = getUnescapedName((ASTNode)ast.getChild(0)); + String tableName = getUnescapedName((ASTNode) ast.getChild(0)); String likeTableName = null; List cols = new ArrayList(); List partCols = new ArrayList(); @@ -8728,7 +8919,7 @@ break; case HiveParser.TOK_LIKETABLE: if (child.getChildCount() > 0) { - likeTableName = getUnescapedName((ASTNode)child.getChild(0)); + likeTableName = getUnescapedName((ASTNode) child.getChild(0)); if (likeTableName != null) { if (command_type == CTAS) { throw new SemanticException(ErrorMsg.CTAS_CTLT_COEXISTENCE @@ -8852,8 +9043,10 @@ tblProps = addDefaultProperties(tblProps); crtTblDesc = new CreateTableDesc(tableName, isExt, cols, partCols, - bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim, rowFormatParams.fieldEscape, - rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim, comment, + bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim, + rowFormatParams.fieldEscape, + rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim, + comment, storageFormat.inputFormat, storageFormat.outputFormat, location, shared.serde, storageFormat.storageHandler, shared.serdeProps, tblProps, ifNotExists, skewedColNames, skewedValues); @@ -8883,7 +9076,7 @@ try { Table dumpTable = db.newTable(tableName); databaseName = dumpTable.getDbName(); - if (null == db.getDatabase(dumpTable.getDbName()) ) { + if (null == db.getDatabase(dumpTable.getDbName())) { throw new SemanticException(ErrorMsg.DATABASE_NOT_EXISTS.getMsg(dumpTable.getDbName())); } if (null != db.getTable(dumpTable.getDbName(), dumpTable.getTableName(), false)) { @@ -8896,9 +9089,12 @@ tblProps = addDefaultProperties(tblProps); crtTblDesc = new CreateTableDesc(databaseName, tableName, isExt, cols, partCols, - bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim, rowFormatParams.fieldEscape, - rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim, comment, storageFormat.inputFormat, - storageFormat.outputFormat, location, shared.serde, storageFormat.storageHandler, shared.serdeProps, + bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim, + rowFormatParams.fieldEscape, + rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim, + comment, storageFormat.inputFormat, + storageFormat.outputFormat, location, shared.serde, storageFormat.storageHandler, + shared.serdeProps, tblProps, ifNotExists, skewedColNames, skewedValues); crtTblDesc.setStoredAsSubDirectories(storedAsDirs); qb.setTableDesc(crtTblDesc); @@ -8914,7 +9110,7 @@ private ASTNode analyzeCreateView(ASTNode ast, QB qb) throws SemanticException { - String tableName = getUnescapedName((ASTNode)ast.getChild(0)); + String tableName = getUnescapedName((ASTNode) ast.getChild(0)); List cols = null; boolean ifNotExists = false; boolean orReplace = false; @@ -8955,12 +9151,12 @@ } } - if (ifNotExists && orReplace){ + if (ifNotExists && orReplace) { throw new SemanticException("Can't combine IF NOT EXISTS and OR REPLACE."); } createVwDesc = new CreateViewDesc( - tableName, cols, comment, tblProps, partColNames, ifNotExists, orReplace); + tableName, cols, comment, tblProps, partColNames, ifNotExists, orReplace); unparseTranslator.enable(); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), createVwDesc), conf)); @@ -8970,7 +9166,7 @@ private void decideExecMode(List> rootTasks, Context ctx, GlobalLimitCtx globalLimitCtx) - throws SemanticException { + throws SemanticException { // bypass for explain queries for now if (ctx.getExplain()) { @@ -8984,20 +9180,20 @@ } final Context lCtx = ctx; - PathFilter p = new PathFilter () { - public boolean accept(Path file) { - return !lCtx.isMRTmpFileURI(file.toUri().getPath()); - } - }; + PathFilter p = new PathFilter() { + public boolean accept(Path file) { + return !lCtx.isMRTmpFileURI(file.toUri().getPath()); + } + }; List mrtasks = Utilities.getMRTasks(rootTasks); // map-reduce jobs will be run locally based on data size // first find out if any of the jobs needs to run non-locally boolean hasNonLocalJob = false; - for (ExecDriver mrtask: mrtasks) { + for (ExecDriver mrtask : mrtasks) { try { ContentSummary inputSummary = Utilities.getInputSummary - (ctx, (MapredWork)mrtask.getWork(), p); + (ctx, (MapredWork) mrtask.getWork(), p); int numReducers = getNumberOfReducers(mrtask.getWork(), conf); long estimatedInput; @@ -9020,8 +9216,8 @@ if (LOG.isDebugEnabled()) { LOG.debug("Task: " + mrtask.getId() + ", Summary: " + - inputSummary.getLength() + "," + inputSummary.getFileCount() + "," - + numReducers + ", estimated Input: " + estimatedInput); + inputSummary.getLength() + "," + inputSummary.getFileCount() + "," + + numReducers + ", estimated Input: " + estimatedInput); } if (MapRedTask.isEligibleForLocalMode(conf, numReducers, @@ -9032,15 +9228,15 @@ mrtask.setLocalMode(true); } } catch (IOException e) { - throw new SemanticException (e); + throw new SemanticException(e); } } - if(!hasNonLocalJob) { + if (!hasNonLocalJob) { // Entire query can be run locally. // Save the current tracker value and restore it when done. ctx.setOriginalTracker(ShimLoader.getHadoopShims().getJobLauncherRpcAddress(conf)); - ShimLoader.getHadoopShims().setJobLauncherRpcAddress(conf,"local"); + ShimLoader.getHadoopShims().setJobLauncherRpcAddress(conf, "local"); console.printInfo("Automatically selecting local only mode for query"); // If all the tasks can be run locally, we can use local disk for Index: ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java (revision 1425241) +++ ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java (working copy) @@ -291,6 +291,16 @@ HIVE_GROUPING_SETS_EXPR_NOT_IN_GROUPBY(10213, "Grouping sets expression is not in GROUP BY key"), INVALID_PARTITION_SPEC(10214, "Invalid partition spec specified"), + HIVE_GROUPING_SETS_THRESHOLD_NOT_ALLOWED_WITH_SKEW(10215, + "An additional MR job is introduced since the number of rows created per input row " + + "due to grouping sets is more than hive.new.job.grouping.set.cardinality. There is no need " + + "to handle skew separately. set hive.groupby.skewindata to false."), + HIVE_GROUPING_SETS_THRESHOLD_NOT_ALLOWED_WITH_DISTINCTS(10216, + "An additional MR job is introduced since the cardinality of grouping sets " + + "is more than hive.new.job.grouping.set.cardinality. This functionality is not supported " + + "with distincts. Either set hive.new.job.grouping.set.cardinality to a high number " + + "(higher than the number of rows per input row due to grouping sets in the query), or " + + "rewrite the query to not use distincts."), SCRIPT_INIT_ERROR(20000, "Unable to initialize custom script."), SCRIPT_IO_ERROR(20001, "An error occurred while reading or writing to your custom script. "