Zhenxiao Luo created HIVE-3837: ---------------------------------- Summary: Three Table BucketMapJoin is failing Key: HIVE-3837 URL: https://issues.apache.org/jira/browse/HIVE-3837 Project: Hive Issue Type: Bug Reporter: Zhenxiao Luo Assignee: Zhenxiao Luo
The following testcase shows that three table BucketMapJoin is failing: set hive.optimize.bucketmapjoin = true; CREATE TABLE t1 (key1 int, value1 string) partitioned by (ds1 string) CLUSTERED BY (key1) INTO 2 BUCKETS STORED AS TEXTFILE; load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds1='part1'); load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds1='part1'); load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds1='part2'); load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds1='part2'); CREATE TABLE t2 (key2 int, value2 string) partitioned by (ds2 string) CLUSTERED BY (key2) INTO 2 BUCKETS STORED AS TEXTFILE; load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t2 partition(ds2='part1'); load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t2 partition(ds2='part1'); load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t2 partition(ds2='part2'); load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t2 partition(ds2='part2'); CREATE TABLE t3 (key3 int, value3 string) partitioned by (ds3 string) CLUSTERED BY (key3) INTO 2 BUCKETS STORED AS TEXTFILE; load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t3 partition(ds3='part1'); load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t3 partition(ds3='part1'); -- Three Tables Join explain extended select /*+mapjoin(b,c)*/ count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3); select /*+mapjoin(b,c)*/ count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3); select count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3); The result is: PREHOOK: query: CREATE TABLE t1 (key1 int, value1 string) partitioned by (ds1 string) CLUSTERED BY (key1) INTO 2 BUCKETS STORED AS TEXTFILE PREHOOK: type: CREATETABLE POSTHOOK: query: CREATE TABLE t1 (key1 int, value1 string) partitioned by (ds1 string) CLUSTERED BY (key1) INTO 2 BUCKETS STORED AS TEXTFILE POSTHOOK: type: CREATETABLE POSTHOOK: Output: default@t1 PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds1='part1') PREHOOK: type: LOAD PREHOOK: Output: default@t1 POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds1='part1') POSTHOOK: type: LOAD POSTHOOK: Output: default@t1 POSTHOOK: Output: default@t1@ds1=part1 PREHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds1='part1') PREHOOK: type: LOAD PREHOOK: Output: default@t1@ds1=part1 POSTHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds1='part1') POSTHOOK: type: LOAD POSTHOOK: Output: default@t1@ds1=part1 PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds1='part2') PREHOOK: type: LOAD PREHOOK: Output: default@t1 POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds1='part2') POSTHOOK: type: LOAD POSTHOOK: Output: default@t1 POSTHOOK: Output: default@t1@ds1=part2 PREHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds1='part2') PREHOOK: type: LOAD PREHOOK: Output: default@t1@ds1=part2 POSTHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds1='part2') POSTHOOK: type: LOAD POSTHOOK: Output: default@t1@ds1=part2 PREHOOK: query: CREATE TABLE t2 (key2 int, value2 string) partitioned by (ds2 string) CLUSTERED BY (key2) INTO 2 BUCKETS STORED AS TEXTFILE PREHOOK: type: CREATETABLE POSTHOOK: query: CREATE TABLE t2 (key2 int, value2 string) partitioned by (ds2 string) CLUSTERED BY (key2) INTO 2 BUCKETS STORED AS TEXTFILE POSTHOOK: type: CREATETABLE POSTHOOK: Output: default@t2 PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t2 partition(ds2='part1') PREHOOK: type: LOAD PREHOOK: Output: default@t2 POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t2 partition(ds2='part1') POSTHOOK: type: LOAD POSTHOOK: Output: default@t2 POSTHOOK: Output: default@t2@ds2=part1 PREHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t2 partition(ds2='part1') PREHOOK: type: LOAD PREHOOK: Output: default@t2@ds2=part1 POSTHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t2 partition(ds2='part1') POSTHOOK: type: LOAD POSTHOOK: Output: default@t2@ds2=part1 PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t2 partition(ds2='part2') PREHOOK: type: LOAD PREHOOK: Output: default@t2 POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t2 partition(ds2='part2') POSTHOOK: type: LOAD POSTHOOK: Output: default@t2 POSTHOOK: Output: default@t2@ds2=part2 PREHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t2 partition(ds2='part2') PREHOOK: type: LOAD PREHOOK: Output: default@t2@ds2=part2 POSTHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t2 partition(ds2='part2') POSTHOOK: type: LOAD POSTHOOK: Output: default@t2@ds2=part2 PREHOOK: query: CREATE TABLE t3 (key3 int, value3 string) partitioned by (ds3 string) CLUSTERED BY (key3) INTO 2 BUCKETS STORED AS TEXTFILE PREHOOK: type: CREATETABLE POSTHOOK: query: CREATE TABLE t3 (key3 int, value3 string) partitioned by (ds3 string) CLUSTERED BY (key3) INTO 2 BUCKETS STORED AS TEXTFILE POSTHOOK: type: CREATETABLE POSTHOOK: Output: default@t3 PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t3 partition(ds3='part1') PREHOOK: type: LOAD PREHOOK: Output: default@t3 POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t3 partition(ds3='part1') POSTHOOK: type: LOAD POSTHOOK: Output: default@t3 POSTHOOK: Output: default@t3@ds3=part1 PREHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t3 partition(ds3='part1') PREHOOK: type: LOAD PREHOOK: Output: default@t3@ds3=part1 POSTHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t3 partition(ds3='part1') POSTHOOK: type: LOAD POSTHOOK: Output: default@t3@ds3=part1 PREHOOK: query: -- Three Tables Join explain extended select /*+mapjoin(b,c)*/ count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3) PREHOOK: type: QUERY POSTHOOK: query: -- Three Tables Join explain extended select /*+mapjoin(b,c)*/ count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3) POSTHOOK: type: QUERY ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1) a) (TOK_TABREF (TOK_TABNAME t2) b) (and (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key2)) (= (. (TOK_TABLE_OR_COL a) ds1) (. (TOK_TABLE_OR_COL b) ds2)))) (TOK_TABREF (TOK_TABNAME t3) c) (and (= (. (TOK_TABLE_OR_COL b) key2) (. (TOK_TABLE_OR_COL c) key3)) (= (. (TOK_TABLE_OR_COL b) ds2) (. (TOK_TABLE_OR_COL c) ds3))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b c))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: Stage-5 is a root stage Stage-1 depends on stages: Stage-5 Stage-2 depends on stages: Stage-1 Stage-0 is a root stage STAGE PLANS: Stage: Stage-5 Map Reduce Local Work Alias -> Map Local Tables: b Fetch Operator limit: -1 c Fetch Operator limit: -1 Alias -> Map Local Operator Tree: b TableScan alias: b GatherStats: false HashTable Sink Operator condition expressions: 0 1 2 handleSkewJoin: false keys: 0 [Column[key1], Column[ds1]] 1 [Column[key2], Column[ds2]] 2 [Column[key3], Column[ds3]] Position of Big Table: 0 c TableScan alias: c GatherStats: false HashTable Sink Operator condition expressions: 0 1 2 handleSkewJoin: false keys: 0 [Column[key1], Column[ds1]] 1 [Column[key2], Column[ds2]] 2 [Column[key3], Column[ds3]] Position of Big Table: 0 Bucket Mapjoin Context: Alias Bucket Base File Name Mapping: b {ds1=part1/srcbucket20.txt=[ds2=part1/srcbucket20.txt, ds2=part2/srcbucket20.txt], ds1=part1/srcbucket21.txt=[ds2=part1/srcbucket21.txt, ds2=part2/srcbucket21.txt], ds1=part2/srcbucket20.txt=[ds2=part1/srcbucket20.txt, ds2=part2/srcbucket20.txt], ds1=part2/srcbucket21.txt=[ds2=part1/srcbucket21.txt, ds2=part2/srcbucket21.txt]} c {ds1=part1/srcbucket20.txt=[ds3=part1/srcbucket20.txt], ds1=part1/srcbucket21.txt=[ds3=part1/srcbucket21.txt], ds1=part2/srcbucket20.txt=[ds3=part1/srcbucket20.txt], ds1=part2/srcbucket21.txt=[ds3=part1/srcbucket21.txt]} Alias Bucket File Name Mapping: #### A masked pattern was here #### Alias Bucket Output File Name Mapping: #### A masked pattern was here #### Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: a TableScan alias: a GatherStats: false Map Join Operator condition map: Inner Join 0 to 1 Inner Join 1 to 2 condition expressions: 0 1 2 handleSkewJoin: false keys: 0 [Column[key1], Column[ds1]] 1 [Column[key2], Column[ds2]] 2 [Column[key3], Column[ds3]] Position of Big Table: 0 File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: columns columns.types escape.delim \ TotalFiles: 1 GatherStats: false MultiFileSpray: false Local Work: Map Reduce Local Work Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: #### A masked pattern was here #### Partition base file name: ds1=part1 input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat partition values: ds1 part1 properties: bucket_count 2 bucket_field_name key1 columns key1,value1 columns.types int:string #### A masked pattern was here #### name default.t1 numFiles 2 numPartitions 2 numRows 0 partition_columns ds1 rawDataSize 0 serialization.ddl struct t1 { i32 key1, string value1} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 2750 #### A masked pattern was here #### 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 2 bucket_field_name key1 columns key1,value1 columns.types int:string #### A masked pattern was here #### name default.t1 numFiles 4 numPartitions 2 numRows 0 partition_columns ds1 rawDataSize 0 serialization.ddl struct t1 { i32 key1, string value1} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 5500 #### A masked pattern was here #### serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 #### A masked pattern was here #### Partition base file name: ds1=part2 input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat partition values: ds1 part2 properties: bucket_count 2 bucket_field_name key1 columns key1,value1 columns.types int:string #### A masked pattern was here #### name default.t1 numFiles 2 numPartitions 2 numRows 0 partition_columns ds1 rawDataSize 0 serialization.ddl struct t1 { i32 key1, string value1} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 2750 #### A masked pattern was here #### 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 2 bucket_field_name key1 columns key1,value1 columns.types int:string #### A masked pattern was here #### name default.t1 numFiles 4 numPartitions 2 numRows 0 partition_columns ds1 rawDataSize 0 serialization.ddl struct t1 { i32 key1, string value1} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe totalSize 5500 #### A masked pattern was here #### serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.t1 name: default.t1 Truncated Path -> Alias: /t1/ds1=part1 [a] /t1/ds1=part2 [a] Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: #### A masked pattern was here #### Select Operator Select Operator Group By Operator aggregations: expr: count() bucketGroup: false mode: hash outputColumnNames: _col0 Reduce Output Operator sort order: tag: -1 value expressions: expr: _col0 type: bigint Needs Tagging: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: #### A masked pattern was here #### Partition base file name: -mr-10002 input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: columns columns.types escape.delim \ input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: columns columns.types escape.delim \ Reduce Operator Tree: Group By Operator aggregations: expr: count(VALUE._col0) bucketGroup: false mode: mergepartial outputColumnNames: _col0 Select Operator expressions: expr: _col0 type: bigint outputColumnNames: _col0 File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: columns _col0 columns.types bigint escape.delim \ serialization.format 1 TotalFiles: 1 GatherStats: false MultiFileSpray: false Truncated Path -> Alias: #### A masked pattern was here #### Stage: Stage-0 Fetch Operator limit: -1 PREHOOK: query: select /*+mapjoin(b,c)*/ count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3) PREHOOK: type: QUERY PREHOOK: Input: default@t1@ds1=part1 PREHOOK: Input: default@t1@ds1=part2 PREHOOK: Input: default@t2@ds2=part1 PREHOOK: Input: default@t2@ds2=part2 PREHOOK: Input: default@t3@ds3=part1 #### A masked pattern was here #### POSTHOOK: query: select /*+mapjoin(b,c)*/ count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3) POSTHOOK: type: QUERY POSTHOOK: Input: default@t1@ds1=part1 POSTHOOK: Input: default@t1@ds1=part2 POSTHOOK: Input: default@t2@ds2=part1 POSTHOOK: Input: default@t2@ds2=part2 POSTHOOK: Input: default@t3@ds3=part1 #### A masked pattern was here #### 0 PREHOOK: query: select count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3) PREHOOK: type: QUERY PREHOOK: Input: default@t1@ds1=part1 PREHOOK: Input: default@t1@ds1=part2 PREHOOK: Input: default@t2@ds2=part1 PREHOOK: Input: default@t2@ds2=part2 PREHOOK: Input: default@t3@ds3=part1 #### A masked pattern was here #### POSTHOOK: query: select count(*) from t1 a join t2 b on (a.key1=b.key2 and a.ds1=b.ds2) join t3 c on (b.key2=c.key3 and b.ds2=c.ds3) POSTHOOK: type: QUERY POSTHOOK: Input: default@t1@ds1=part1 POSTHOOK: Input: default@t1@ds1=part2 POSTHOOK: Input: default@t2@ds2=part1 POSTHOOK: Input: default@t2@ds2=part2 POSTHOOK: Input: default@t3@ds3=part1 #### A masked pattern was here #### 1114 We could see that Three Table BucketMapJoin's result is incorrect(should be 1114 instead of 0). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira