[ https://issues.apache.org/jira/browse/HIVE-16600?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
liyunzhang_intel updated HIVE-16600: ------------------------------------ Attachment: HIVE-16600.7.patch [~lirui]: thanks for your review. update HIVE-16600.7.patch. Changes: add two test case, 1. order by limit + multi-insert {code} FROM (select key,value from src order by key limit 10) a INSERT OVERWRITE TABLE e1 SELECT key, value INSERT OVERWRITE TABLE e2 SELECT key; {code} we should disable parallel order by in this case, which matches expectation {code} Stage-2 is a root stage [MAPRED] Stage-0 depends on stages: Stage-2 [MOVE] Stage-3 depends on stages: Stage-0 [STATS] Stage-1 depends on stages: Stage-2 [MOVE] Stage-4 depends on stages: Stage-1 [STATS] STAGE PLANS: Stage: Stage-2 Spark Edges: Reducer 2 <- Map 1 (SORT, 1) DagName: root_20170519161033_283bdbba-0f73-4db1-afba-4da456e3cc55:1 Vertices: Map 1 Map Operator Tree: TableScan alias: src Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), value (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: a sort order: + Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE tag: -1 TopN: 10 TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: string) auto parallelism: false Path -> Alias: hdfs://bdpe41:8020/user/hive/warehouse/src [src] Path -> Partition: hdfs://bdpe41:8020/user/hive/warehouse/src Partition base file name: src input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: bucket_count -1 column.name.delimiter , columns key,value columns.comments 'default','default' columns.types string:string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/src name default.src numFiles 1 numRows 0 rawDataSize 0 serialization.ddl struct src { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 5812 transient_lastDdlTime 1493960133 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: bucket_count -1 column.name.delimiter , columns key,value columns.comments 'default','default' columns.types string:string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/src name default.src numFiles 1 numRows 0 rawDataSize 0 serialization.ddl struct src { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 5812 transient_lastDdlTime 1493960133 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src Truncated Path -> Alias: /src [src] Reducer 2 Needs Tagging: false Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 directory: hdfs://bdpe41:8020/user/hive/warehouse/e1/.hive-staging_hive_2017-05-19_16-10-33_636_2374207122339693520-1/-ext-10000 NumFilesPerFileSink: 1 Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE Stats Publishing Key Prefix: hdfs://bdpe41:8020/user/hive/warehouse/e1/.hive-staging_hive_2017-05-19_16-10-33_636_2374207122339693520-1/-ext-10000/ table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} bucket_count -1 column.name.delimiter , columns key,value columns.comments columns.types string:string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/e1 name default.e1 numFiles 0 numRows 0 rawDataSize 0 serialization.ddl struct e1 { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 0 transient_lastDdlTime 1495181433 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.e1 TotalFiles: 1 GatherStats: true MultiFileSpray: false Select Operator expressions: _col0 (type: string) outputColumnNames: _col0 Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 2 directory: hdfs://bdpe41:8020/user/hive/warehouse/e2/.hive-staging_hive_2017-05-19_16-10-33_636_2374207122339693520-1/-ext-10002 NumFilesPerFileSink: 1 Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE Stats Publishing Key Prefix: hdfs://bdpe41:8020/user/hive/warehouse/e2/.hive-staging_hive_2017-05-19_16-10-33_636_2374207122339693520-1/-ext-10002/ table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} bucket_count -1 column.name.delimiter , columns key columns.comments columns.types string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/e2 name default.e2 numFiles 0 numRows 0 rawDataSize 0 serialization.ddl struct e2 { string key} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 0 transient_lastDdlTime 1495181433 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.e2 TotalFiles: 1 GatherStats: true MultiFileSpray: false Stage: Stage-0 Move Operator tables: replace: true source: hdfs://bdpe41:8020/user/hive/warehouse/e1/.hive-staging_hive_2017-05-19_16-10-33_636_2374207122339693520-1/-ext-10000 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} bucket_count -1 column.name.delimiter , columns key,value columns.comments columns.types string:string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/e1 name default.e1 numFiles 0 numRows 0 rawDataSize 0 serialization.ddl struct e1 { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 0 transient_lastDdlTime 1495181433 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.e1 Stage: Stage-3 Stats-Aggr Operator Stats Aggregation Key Prefix: hdfs://bdpe41:8020/user/hive/warehouse/e1/.hive-staging_hive_2017-05-19_16-10-33_636_2374207122339693520-1/-ext-10000/ Stage: Stage-1 Move Operator tables: replace: true source: hdfs://bdpe41:8020/user/hive/warehouse/e2/.hive-staging_hive_2017-05-19_16-10-33_636_2374207122339693520-1/-ext-10002 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} bucket_count -1 column.name.delimiter , columns key columns.comments columns.types string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/e2 name default.e2 numFiles 0 numRows 0 rawDataSize 0 serialization.ddl struct e2 { string key} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 0 transient_lastDdlTime 1495181433 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.e2 Stage: Stage-4 Stats-Aggr Operator Stats Aggregation Key Prefix: hdfs://bdpe41:8020/user/hive/warehouse/e2/.hive-staging_hive_2017-05-19_16-10-33_636_2374207122339693520-1/-ext-10002/ {code} 2. orderby + multi-insert( 2 limit in sub query) We should enable parallel order by, which matches expectation. {code} FROM (select key,value from src order by key) a INSERT OVERWRITE TABLE e1 SELECT key, value limit 10 INSERT OVERWRITE TABLE e2 SELECT key limit 5; {code} explain {code} Stage-0 depends on stages: Stage-2 [MOVE] Stage-3 depends on stages: Stage-0 [STATS] Stage-1 depends on stages: Stage-2 [MOVE] Stage-4 depends on stages: Stage-1 [STATS] STAGE PLANS: Stage: Stage-2 Spark Edges: Reducer 3 <- Reducer 5 (GROUP, 1) Reducer 4 <- Reducer 6 (GROUP, 1) Reducer 5 <- Map 1 (SORT, 46) Reducer 6 <- Map 1 (SORT, 46) DagName: root_20170519162519_7384dca1-7dc7-4186-99ae-d021eda761a6:1 Vertices: Map 1 Map Operator Tree: TableScan alias: src Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), value (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: a sort order: + Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string) auto parallelism: false Path -> Alias: hdfs://bdpe41:8020/user/hive/warehouse/src [src] Path -> Partition: hdfs://bdpe41:8020/user/hive/warehouse/src Partition base file name: src input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: bucket_count -1 column.name.delimiter , columns key,value columns.comments 'default','default' columns.types string:string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/src name default.src numFiles 1 numRows 0 rawDataSize 0 serialization.ddl struct src { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 5812 transient_lastDdlTime 1493960133 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: bucket_count -1 column.name.delimiter , columns key,value columns.comments 'default','default' columns.types string:string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/src name default.src numFiles 1 numRows 0 rawDataSize 0 serialization.ddl struct src { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 5812 transient_lastDdlTime 1493960133 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src Truncated Path -> Alias: /src [src] Reducer 3 Needs Tagging: false Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: string), VALUE._col1 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 directory: hdfs://bdpe41:8020/user/hive/warehouse/e1/.hive-staging_hive_2017-05-19_16-25-19_345_5343780662632806715-1/-ext-10000 NumFilesPerFileSink: 1 Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE Stats Publishing Key Prefix: hdfs://bdpe41:8020/user/hive/warehouse/e1/.hive-staging_hive_2017-05-19_16-25-19_345_5343780662632806715-1/-ext-10000/ table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} bucket_count -1 column.name.delimiter , columns key,value columns.comments columns.types string:string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/e1 name default.e1 numFiles 0 numRows 0 rawDataSize 0 serialization.ddl struct e1 { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 0 transient_lastDdlTime 1495182319 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.e1 TotalFiles: 1 GatherStats: true MultiFileSpray: false Reducer 4 Needs Tagging: false Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: string) outputColumnNames: _col0 Statistics: Num rows: 5 Data size: 1000 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 5 Statistics: Num rows: 5 Data size: 1000 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 2 directory: hdfs://bdpe41:8020/user/hive/warehouse/e2/.hive-staging_hive_2017-05-19_16-25-19_345_5343780662632806715-1/-ext-10002 NumFilesPerFileSink: 1 Statistics: Num rows: 5 Data size: 1000 Basic stats: COMPLETE Column stats: NONE Stats Publishing Key Prefix: hdfs://bdpe41:8020/user/hive/warehouse/e2/.hive-staging_hive_2017-05-19_16-25-19_345_5343780662632806715-1/-ext-10002/ table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} bucket_count -1 column.name.delimiter , columns key columns.comments columns.types string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/e2 name default.e2 numFiles 0 numRows 0 rawDataSize 0 serialization.ddl struct e2 { string key} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 0 transient_lastDdlTime 1495182319 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.e2 TotalFiles: 1 GatherStats: true MultiFileSpray: false Reducer 5 Needs Tagging: false Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE tag: -1 TopN: 10 TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string), _col1 (type: string) auto parallelism: false Reducer 6 Needs Tagging: false Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) outputColumnNames: _col0 Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 5 Statistics: Num rows: 5 Data size: 1000 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: Statistics: Num rows: 5 Data size: 1000 Basic stats: COMPLETE Column stats: NONE tag: -1 TopN: 5 TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string) auto parallelism: false Stage: Stage-0 Move Operator tables: replace: true source: hdfs://bdpe41:8020/user/hive/warehouse/e1/.hive-staging_hive_2017-05-19_16-25-19_345_5343780662632806715-1/-ext-10000 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} bucket_count -1 column.name.delimiter , columns key,value columns.comments columns.types string:string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/e1 name default.e1 numFiles 0 numRows 0 rawDataSize 0 serialization.ddl struct e1 { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 0 transient_lastDdlTime 1495182319 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.e1 Stage: Stage-3 Stats-Aggr Operator Stats Aggregation Key Prefix: hdfs://bdpe41:8020/user/hive/warehouse/e1/.hive-staging_hive_2017-05-19_16-25-19_345_5343780662632806715-1/-ext-10000/ Stage: Stage-1 Move Operator tables: replace: true source: hdfs://bdpe41:8020/user/hive/warehouse/e2/.hive-staging_hive_2017-05-19_16-25-19_345_5343780662632806715-1/-ext-10002 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} bucket_count -1 column.name.delimiter , columns key columns.comments columns.types string file.inputformat org.apache.hadoop.mapred.TextInputFormat file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat location hdfs://bdpe41:8020/user/hive/warehouse/e2 name default.e2 numFiles 0 numRows 0 rawDataSize 0 serialization.ddl struct e2 { string key} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 0 transient_lastDdlTime 1495182319 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.e2 Stage: Stage-4 Stats-Aggr Operator Stats Aggregation Key Prefix: hdfs://bdpe41:8020/user/hive/warehouse/e2/.hive-staging_hive_2017-05-19_16-25-19_345_5343780662632806715-1/-ext-10002/ {code} > Refactor SetSparkReducerParallelism#needSetParallelism to enable parallel > order by in multi_insert cases > -------------------------------------------------------------------------------------------------------- > > Key: HIVE-16600 > URL: https://issues.apache.org/jira/browse/HIVE-16600 > Project: Hive > Issue Type: Sub-task > Reporter: liyunzhang_intel > Assignee: liyunzhang_intel > Attachments: HIVE-16600.1.patch, HIVE-16600.2.patch, > HIVE-16600.3.patch, HIVE-16600.4.patch, HIVE-16600.5.patch, > HIVE-16600.6.patch, HIVE-16600.7.patch, mr.explain, mr.explain.log.HIVE-16600 > > > multi_insert_gby.case.q > {code} > set hive.exec.reducers.bytes.per.reducer=256; > set hive.optimize.sampling.orderby=true; > drop table if exists e1; > drop table if exists e2; > create table e1 (key string, value string); > create table e2 (key string); > FROM (select key, cast(key as double) as keyD, value from src order by key) a > INSERT OVERWRITE TABLE e1 > SELECT key, value > INSERT OVERWRITE TABLE e2 > SELECT key; > select * from e1; > select * from e2; > {code} > the parallelism of Sort is 1 even we enable parallel order > by("hive.optimize.sampling.orderby" is set as "true"). This is not > reasonable because the parallelism should be calcuated by > [Utilities.estimateReducers|https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java#L170] > this is because SetSparkReducerParallelism#needSetParallelism returns false > when [children size of > RS|https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java#L207] > is greater than 1. > in this case, the children size of {{RS[2]}} is two. > the logical plan of the case > {code} > TS[0]-SEL[1]-RS[2]-SEL[3]-SEL[4]-FS[5] > -SEL[6]-FS[7] > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346)