Index: build-common.xml =================================================================== --- build-common.xml (revision 1380935) +++ build-common.xml (working copy) @@ -57,7 +57,7 @@ - + Index: ql/src/test/results/clientpositive/bucketmapjoin7.q.out =================================================================== --- ql/src/test/results/clientpositive/bucketmapjoin7.q.out (revision 0) +++ ql/src/test/results/clientpositive/bucketmapjoin7.q.out (revision 0) @@ -0,0 +1,218 @@ +PREHOOK: query: CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@srcbucket_mapjoin_part_1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: LOAD +PREHOOK: Output: default@srcbucket_mapjoin_part_1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@srcbucket_mapjoin_part_1 +POSTHOOK: Output: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: LOAD +PREHOOK: Output: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +PREHOOK: query: CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@srcbucket_mapjoin_part_2 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: LOAD +PREHOOK: Output: default@srcbucket_mapjoin_part_2 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@srcbucket_mapjoin_part_2 +POSTHOOK: Output: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: LOAD +PREHOOK: Output: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +PREHOOK: query: -- Tests that bucket map join works with a table with more than one level of partitioning + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' LIMIT 1 +PREHOOK: type: QUERY +POSTHOOK: query: -- Tests that bucket map join works with a table with more than one level of partitioning + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' LIMIT 1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (AND (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL b) ds) '2008-04-08')))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_LIMIT 1))) + +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-1 depends on stages: Stage-3 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-3 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + GatherStats: false + HashTable Sink Operator + condition expressions: + 0 {key} + 1 {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds=2008-04-08/hr=0/srcbucket20.txt=[ds=2008-04-08/hr=0/srcbucket20.txt], ds=2008-04-08/hr=0/srcbucket21.txt=[ds=2008-04-08/hr=0/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 + condition expressions: + 0 {key} + 1 {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col7 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col7 + type: string + outputColumnNames: _col0, _col7 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col7 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types int:string + escape.delim \ + serialization.format 1 + 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: hr=0 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 0 + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_1 + numFiles 2 + numPartitions 1 + numRows 0 + partition_columns ds/hr + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} + 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 key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_1 + numFiles 2 + numPartitions 1 + numRows 0 + partition_columns ds/hr + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} + 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 + name: default.srcbucket_mapjoin_part_1 + name: default.srcbucket_mapjoin_part_1 + + Stage: Stage-0 + Fetch Operator + limit: 1 + + +PREHOOK: query: SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' LIMIT 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +#### A masked pattern was here #### +POSTHOOK: query: SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' LIMIT 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +POSTHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +#### A masked pattern was here #### +27 val_27 Index: ql/src/test/queries/clientpositive/bucketmapjoin7.q =================================================================== --- ql/src/test/queries/clientpositive/bucketmapjoin7.q (revision 0) +++ ql/src/test/queries/clientpositive/bucketmapjoin7.q (revision 0) @@ -0,0 +1,24 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); + +CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); + +set hive.optimize.bucketmapjoin=true; + +-- Tests that bucket map join works with a table with more than one level of partitioning + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' LIMIT 1; + +SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' LIMIT 1; Index: ql/src/java/org/apache/hadoop/hive/ql/plan/BucketMapJoinContext.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/BucketMapJoinContext.java (revision 1380935) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/BucketMapJoinContext.java (working copy) @@ -17,9 +17,6 @@ */ package org.apache.hadoop.hive.ql.plan; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.BucketMatcher; - import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; @@ -28,6 +25,10 @@ import java.util.Map; import java.util.regex.Pattern; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.ql.exec.BucketMatcher; + /** * was inner class of MapreLocalWork. context for bucket mapjoin (or smb join) */ @@ -199,7 +200,8 @@ Map mapping = inputToPartSpecMapping == null ? inputToPartSpecMapping = revert(bigTablePartSpecToFileMapping) : inputToPartSpecMapping; String partSpec = mapping.get(inputPath); - return partSpec == null || partSpec.isEmpty() ? fileName : "(" + partSpec + ")" + fileName; + return partSpec == null || partSpec.isEmpty() ? fileName : + "(" + FileUtils.escapePathName(partSpec) + ")" + fileName; } // revert partSpecToFileMapping to inputToPartSpecMapping