diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java index d9354f3..90295d5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.UnionOperator; @@ -188,19 +189,31 @@ public class BucketMapJoinOptimizer implements Transform { } MapJoinDesc mjDesc = mapJoinOp.getConf(); - LinkedHashMap> aliasToPartitionBucketNumberMapping = - new LinkedHashMap>(); - LinkedHashMap>> aliasToPartitionBucketFileNamesMapping = - new LinkedHashMap>>(); + + // (alias to ) + // AND (alias to ) + // one pair for each partition + // partition key/values info is needed in optimization + LinkedHashMap>> + aliasToPartitionBucketNumberMapping = + new LinkedHashMap>>(); + LinkedHashMap>>> + aliasToPartitionBucketFileNamesMapping = + new LinkedHashMap>>>(); Map> topOps = this.pGraphContext.getTopOps(); Map topToTable = this.pGraphContext.getTopToTable(); - // (partition to bucket file names) and (partition to bucket number) for - // the big table; - LinkedHashMap> bigTblPartsToBucketFileNames = new LinkedHashMap>(); - LinkedHashMap bigTblPartsToBucketNumber = new LinkedHashMap(); + // AND + // for the big table + LinkedHashMap> bigTblPartsToBucketFileNames = + new LinkedHashMap>(); + LinkedHashMap bigTblPartsToBucketNumber = + new LinkedHashMap(); + + // The Join Keys of the big table + List bigTblJoinKeys = new ArrayList(); Integer[] orders = null; // accessing order of join cols to bucket cols, should be same boolean bigTablePartitioned = true; @@ -218,6 +231,20 @@ public class BucketMapJoinOptimizer implements Transform { orders = new Integer[keys.size()]; } + if (alias.equals(baseBigAlias)) { + Iterator keyItr = keys.iterator(); + while (keyItr.hasNext()) { + bigTblJoinKeys.add(keyItr.next()); + } + } + + // + // AND + List> partBucketNumberMapping = + new ArrayList>(); + List>> partBucketFileNamesMapping = + new ArrayList>>(); + Table tbl = topToTable.get(tso); if(tbl.isPartitioned()) { PrunedPartitionList prunedParts; @@ -238,12 +265,10 @@ public class BucketMapJoinOptimizer implements Transform { // construct a mapping of (Partition->bucket file names) and (Partition -> bucket number) if (partitions.isEmpty()) { if (!alias.equals(baseBigAlias)) { - aliasToPartitionBucketNumberMapping.put(alias, Arrays.asList()); - aliasToPartitionBucketFileNamesMapping.put(alias, new ArrayList>()); + aliasToPartitionBucketNumberMapping.put(alias, partBucketNumberMapping); + aliasToPartitionBucketFileNamesMapping.put(alias, partBucketFileNamesMapping); } } else { - List buckets = new ArrayList(); - List> files = new ArrayList>(); for (Partition p : partitions) { if (!checkBucketColumns(p.getBucketCols(), keys, orders)) { return false; @@ -262,13 +287,19 @@ public class BucketMapJoinOptimizer implements Transform { bigTblPartsToBucketFileNames.put(p, fileNames); bigTblPartsToBucketNumber.put(p, bucketCount); } else { - files.add(fileNames); - buckets.add(bucketCount); + Map partBucketNumMapping = + new LinkedHashMap(); + partBucketNumMapping.put(p, bucketCount); + partBucketNumberMapping.add(partBucketNumMapping); + Map> partBucketFilesMapping = + new LinkedHashMap>(); + partBucketFilesMapping.put(p, fileNames); + partBucketFileNamesMapping.add(partBucketFilesMapping); } } if (!alias.equals(baseBigAlias)) { - aliasToPartitionBucketNumberMapping.put(alias, buckets); - aliasToPartitionBucketFileNamesMapping.put(alias, files); + aliasToPartitionBucketNumberMapping.put(alias, partBucketNumberMapping); + aliasToPartitionBucketFileNamesMapping.put(alias, partBucketFileNamesMapping); } } } else { @@ -290,8 +321,16 @@ public class BucketMapJoinOptimizer implements Transform { bigTblPartsToBucketNumber.put(null, tbl.getNumBuckets()); bigTablePartitioned = false; } else { - aliasToPartitionBucketNumberMapping.put(alias, Arrays.asList(num)); - aliasToPartitionBucketFileNamesMapping.put(alias, Arrays.asList(fileNames)); + Map partBucketNumMapping = + new LinkedHashMap(); + partBucketNumMapping.put(null, num); + partBucketNumberMapping.add(partBucketNumMapping); + aliasToPartitionBucketNumberMapping.put(alias, partBucketNumberMapping); + Map> partBucketFilesMapping = + new LinkedHashMap>(); + partBucketFilesMapping.put(null, fileNames); + partBucketFileNamesMapping.add(partBucketFilesMapping); + aliasToPartitionBucketFileNamesMapping.put(alias, partBucketFileNamesMapping); } } } @@ -322,29 +361,26 @@ public class BucketMapJoinOptimizer implements Transform { if (alias.equals(baseBigAlias)) { continue; } - for (List names : aliasToPartitionBucketFileNamesMapping.get(alias)) { - Collections.sort(names); + Iterator>> partFilesItr = + aliasToPartitionBucketFileNamesMapping.get(alias).iterator(); + while (partFilesItr.hasNext()) { + Iterator>> entry_itr = + partFilesItr.next().entrySet().iterator(); + while (entry_itr.hasNext()) { + Entry> entry = entry_itr.next(); + Collections.sort(entry.getValue()); + } } - List smallTblBucketNums = aliasToPartitionBucketNumberMapping.get(alias); - List> smallTblFilesList = aliasToPartitionBucketFileNamesMapping.get(alias); - Map> mapping = new LinkedHashMap>(); aliasBucketFileNameMapping.put(alias, mapping); - // for each bucket file in big table, get the corresponding bucket file - // name in the small table. - //more than 1 partition in the big table, do the mapping for each partition - Iterator>> bigTblPartToBucketNames = - bigTblPartsToBucketFileNames.entrySet().iterator(); - Iterator> bigTblPartToBucketNum = bigTblPartsToBucketNumber - .entrySet().iterator(); - while (bigTblPartToBucketNames.hasNext()) { - assert bigTblPartToBucketNum.hasNext(); - int bigTblBucketNum = bigTblPartToBucketNum.next().getValue(); - List bigTblBucketNameList = bigTblPartToBucketNames.next().getValue(); - fillMapping(smallTblBucketNums, smallTblFilesList, - mapping, bigTblBucketNum, bigTblBucketNameList, desc.getBigTableBucketNumMapping()); - } + fillBucketFileMapping(bigTblJoinKeys, alias, + aliasToPartitionBucketNumberMapping, + aliasToPartitionBucketFileNamesMapping, + mapping, + bigTblPartsToBucketNumber, + bigTblPartsToBucketFileNames, + desc.getBigTableBucketNumMapping()); } desc.setAliasBucketFileNameMapping(aliasBucketFileNameMapping); desc.setBigTableAlias(baseBigAlias); @@ -394,53 +430,190 @@ public class BucketMapJoinOptimizer implements Transform { return converted; } - // called for each partition of big table and populates mapping for each file in the partition - private void fillMapping( - List smallTblBucketNums, - List> smallTblFilesList, - Map> mapping, - int bigTblBucketNum, List bigTblBucketNameList, - Map bucketFileNameMapping) { - - for (int bindex = 0; bindex < bigTblBucketNameList.size(); bindex++) { - ArrayList resultFileNames = new ArrayList(); - for (int sindex = 0 ; sindex < smallTblBucketNums.size(); sindex++) { - int smallTblBucketNum = smallTblBucketNums.get(sindex); - List smallTblFileNames = smallTblFilesList.get(sindex); - if (bigTblBucketNum >= smallTblBucketNum) { - // if the big table has more buckets than the current small table, - // use "MOD" to get small table bucket names. For example, if the big - // table has 4 buckets and the small table has 2 buckets, then the - // mapping should be 0->0, 1->1, 2->0, 3->1. - int toAddSmallIndex = bindex % smallTblBucketNum; - resultFileNames.add(smallTblFileNames.get(toAddSmallIndex)); - } else { - int jump = smallTblBucketNum / bigTblBucketNum; - for (int i = bindex; i < smallTblFileNames.size(); i = i + jump) { - resultFileNames.add(smallTblFileNames.get(i)); + // for each bucket file in big table, + // get the corresponding bucket file name in the small table. + private void fillBucketFileMapping( + List bigTblJoinKeys, + String alias, + LinkedHashMap>> + aliasToPartitionBucketNumberMapping, + LinkedHashMap>>> + aliasToPartitionBucketFileNamesMapping, + Map> mapping, + LinkedHashMap bigTblPartsToBucketNumber, + LinkedHashMap> bigTblPartsToBucketFileNames, + Map bucketFileNameMapping) { + + List> smallTblPartBucketNums = + aliasToPartitionBucketNumberMapping.get(alias); + List>> smallTblPartFilesList = + aliasToPartitionBucketFileNamesMapping.get(alias); + + Iterator>> bigTblPartToBucketNames = + bigTblPartsToBucketFileNames.entrySet().iterator(); + Iterator> bigTblPartToBucketNum = + bigTblPartsToBucketNumber.entrySet().iterator(); + + while (bigTblPartToBucketNames.hasNext()) { + assert bigTblPartToBucketNum.hasNext(); + Entry bigTblEntry = bigTblPartToBucketNum.next(); + int bigTblBucketNum = bigTblEntry.getValue(); + Partition bigTablePartition = bigTblEntry.getKey(); + List bigTblBucketNameList = bigTblPartToBucketNames.next().getValue(); + + for (int bindex = 0; bindex < bigTblBucketNameList.size(); bindex++) { + ArrayList resultFileNames = new ArrayList(); + for (int sindex = 0 ; sindex < smallTblPartBucketNums.size(); sindex++) { + + Iterator> entryItr = + smallTblPartBucketNums.get(sindex).entrySet().iterator(); + while (entryItr.hasNext()) { + + Entry partBucketEntry = entryItr.next(); + int smallTblBucketNum = partBucketEntry.getValue(); + Partition smallTablePartition = partBucketEntry.getKey(); + + // if bigTable JoinKeys contains the partition columns + // and tables' partition columns do not match, + // then, the join of these two bucket files will generate no result + // do not put them into the bucket file mapping + if (bigTablePartition != null && + bigTablePartition.getTable() != null && + isJoinKeyContainsPartCols( + bigTblJoinKeys, bigTablePartition.getTable().getPartCols()) && + isPartitionMatch(bigTablePartition, smallTablePartition) == false) { + continue; + } else { + // put the file names into bucket file mapping + Iterator>> entry_itr = + smallTblPartFilesList.get(sindex).entrySet().iterator(); + while (entry_itr.hasNext()) { + Entry> entry = entry_itr.next(); + List smallTblFileNames = entry.getValue(); + if (bigTblBucketNum >= smallTblBucketNum) { + // if the big table has more buckets than the current small table, + // use "MOD" to get small table bucket names. For example, if the big + // table has 4 buckets and the small table has 2 buckets, then the + // mapping should be 0->0, 1->1, 2->0, 3->1. + int toAddSmallIndex = bindex % smallTblBucketNum; + resultFileNames.add(smallTblFileNames.get(toAddSmallIndex)); + } else { + int jump = smallTblBucketNum / bigTblBucketNum; + for (int i = bindex; i < smallTblFileNames.size(); i = i + jump) { + resultFileNames.add(smallTblFileNames.get(i)); + } + } + } + } } } + String inputBigTBLBucket = bigTblBucketNameList.get(bindex); + mapping.put(inputBigTBLBucket, resultFileNames); + bucketFileNameMapping.put(inputBigTBLBucket, bindex); } - String inputBigTBLBucket = bigTblBucketNameList.get(bindex); - mapping.put(inputBigTBLBucket, resultFileNames); - bucketFileNameMapping.put(inputBigTBLBucket, bindex); } } - private boolean checkBucketNumberAgainstBigTable( - Map> aliasToBucketNumber, int bucketNumberInPart) { - for (List bucketNums : aliasToBucketNumber.values()) { - for (int nxt : bucketNums) { - boolean ok = (nxt >= bucketNumberInPart) ? nxt % bucketNumberInPart == 0 - : bucketNumberInPart % nxt == 0; - if (!ok) { - return false; - } + // Does join keys contain partition columns? + private boolean isJoinKeyContainsPartCols( + List joinKeys, + List partCols) { + Iterator colItr = partCols.iterator(); + while (colItr.hasNext()) { + FieldSchema fieldSchema = colItr.next(); + if (joinKeys.contains(fieldSchema.getName()) == false) { + return false; } } return true; } + // Does the two partitions' partition columns match? + private boolean isPartitionMatch( + Partition bigTblPartition, + Partition smallTblPartition) { + + // false if having different number of partition columns + if (bigTblPartition.getTable().getPartCols().size() != + smallTblPartition.getTable().getPartCols().size()) { + return false; + } + + // false if partition column types do not match + List bigTblPartKeyTypes = new ArrayList(); + List smallTblPartKeyTypes = new ArrayList(); + Iterator bigTblItr = bigTblPartition.getTable().getPartCols().iterator(); + Iterator smallTblItr = smallTblPartition.getTable().getPartCols().iterator(); + + while (bigTblItr.hasNext()) { + FieldSchema bigTblSchema = bigTblItr.next(); + String schemaType = bigTblSchema.getType(); + if (!bigTblPartKeyTypes.contains(schemaType)) { + bigTblPartKeyTypes.add(schemaType); + } + } + + while (smallTblItr.hasNext()) { + FieldSchema smallTblSchema = smallTblItr.next(); + String schemaType = smallTblSchema.getType(); + if (!smallTblPartKeyTypes.contains(schemaType)) { + smallTblPartKeyTypes.add(schemaType); + } + } + + if (!bigTblPartKeyTypes.equals(smallTblPartKeyTypes)) { + return false; + } + + // false if partition values diff + List bigTblPartValues = new ArrayList(); + List smallTblPartValues = new ArrayList(); + Iterator bigTblPartValuesItr = bigTblPartition.getValues().iterator(); + Iterator smallTblPartValuesItr = smallTblPartition.getValues().iterator(); + + while(bigTblPartValuesItr.hasNext()) { + String partValue = bigTblPartValuesItr.next(); + if (!bigTblPartValues.contains(partValue)) { + bigTblPartValues.add(partValue); + } + } + + while(smallTblPartValuesItr.hasNext()) { + String partValue = smallTblPartValuesItr.next(); + if (!smallTblPartValues.contains(partValue)) { + smallTblPartValues.add(partValue); + } + } + + if (!bigTblPartValues.equals(smallTblPartValues)) { + return false; + } + + return true; + } + + private boolean checkBucketNumberAgainstBigTable( + Map>> aliasToBucketNumber, int bucketNumberInPart) { + Iterator aliasItr = aliasToBucketNumber.keySet().iterator(); + while(aliasItr.hasNext()) { + String alias = aliasItr.next(); + Iterator> mapItr = aliasToBucketNumber.get(alias).iterator(); + while (mapItr.hasNext()) { + Iterator> partItr = mapItr.next().entrySet().iterator(); + while (partItr.hasNext()) { + Entry partEntry = partItr.next(); + int nxt = partEntry.getValue(); + boolean ok = (nxt >= bucketNumberInPart) ? nxt % bucketNumberInPart == 0 + : bucketNumberInPart % nxt == 0; + if (!ok) { + return false; + } + } + } + } + return true; + } + private List getOnePartitionBucketFileNames(URI location) throws SemanticException { List fileNames = new ArrayList(); diff --git ql/src/test/queries/clientpositive/bucketmapjoin_optimize_partition.q ql/src/test/queries/clientpositive/bucketmapjoin_optimize_partition.q new file mode 100644 index 0000000..369fc27 --- /dev/null +++ ql/src/test/queries/clientpositive/bucketmapjoin_optimize_partition.q @@ -0,0 +1,77 @@ +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/srcbucket22.txt' INTO TABLE t1 partition(ds1='part2'); +load data local inpath '../data/files/srcbucket23.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/srcbucket22.txt' INTO TABLE t2 partition(ds2='part2'); +load data local inpath '../data/files/srcbucket23.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/srcbucket22.txt' INTO TABLE t3 partition(ds3='part1'); + +CREATE TABLE t4 (key4 int, value4 string) partitioned by (ds4 string) CLUSTERED BY (key4) INTO 4 BUCKETS STORED AS TEXTFILE; +load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t4 partition(ds4='part1'); +load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t4 partition(ds4='part1'); +load data local inpath '../data/files/srcbucket22.txt' INTO TABLE t4 partition(ds4='part1'); +load data local inpath '../data/files/srcbucket23.txt' INTO TABLE t4 partition(ds4='part1'); + +set hive.optimize.bucketmapjoin = true; + +explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 and a.ds1=b.ds2; + +select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 and a.ds1=b.ds2; + +explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2; + +select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2; + +explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 and a.ds1=b.ds3; + +select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 and a.ds1=b.ds3; + +explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3; + +select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3; + +explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 and a.ds1=b.ds4; + +select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 and a.ds1=b.ds4; + +explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4; + +select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4; diff --git ql/src/test/results/clientpositive/bucketmapjoin11.q.out ql/src/test/results/clientpositive/bucketmapjoin11.q.out index c7d65d8..c566a92 100644 --- ql/src/test/results/clientpositive/bucketmapjoin11.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin11.q.out @@ -428,7 +428,7 @@ STAGE PLANS: Position of Big Table: 0 Bucket Mapjoin Context: Alias Bucket Base File Name Mapping: - b {part=1/srcbucket20.txt=[part=1/srcbucket20.txt, part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt, part=1/srcbucket23.txt, part=2/srcbucket21.txt], part=2/srcbucket20.txt=[part=1/srcbucket20.txt, part=2/srcbucket20.txt], part=2/srcbucket21.txt=[part=1/srcbucket21.txt, part=2/srcbucket21.txt], part=2/srcbucket22.txt=[part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=2/srcbucket23.txt=[part=1/srcbucket23.txt, part=2/srcbucket21.txt]} + b {part=1/srcbucket20.txt=[part=1/srcbucket20.txt, part=1/srcbucket22.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt, part=1/srcbucket23.txt], part=2/srcbucket20.txt=[part=2/srcbucket20.txt], part=2/srcbucket21.txt=[part=2/srcbucket21.txt], part=2/srcbucket22.txt=[part=2/srcbucket20.txt], part=2/srcbucket23.txt=[part=2/srcbucket21.txt]} Alias Bucket File Name Mapping: #### A masked pattern was here #### Alias Bucket Output File Name Mapping: diff --git ql/src/test/results/clientpositive/bucketmapjoin_optimize_partition.q.out ql/src/test/results/clientpositive/bucketmapjoin_optimize_partition.q.out new file mode 100644 index 0000000..0ef1608 --- /dev/null +++ ql/src/test/results/clientpositive/bucketmapjoin_optimize_partition.q.out @@ -0,0 +1,1758 @@ +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/srcbucket22.txt' INTO TABLE t1 partition(ds1='part2') +PREHOOK: type: LOAD +PREHOOK: Output: default@t1 +POSTHOOK: query: load data local inpath '../data/files/srcbucket22.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/srcbucket23.txt' INTO TABLE t1 partition(ds1='part2') +PREHOOK: type: LOAD +PREHOOK: Output: default@t1@ds1=part2 +POSTHOOK: query: load data local inpath '../data/files/srcbucket23.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/srcbucket22.txt' INTO TABLE t2 partition(ds2='part2') +PREHOOK: type: LOAD +PREHOOK: Output: default@t2 +POSTHOOK: query: load data local inpath '../data/files/srcbucket22.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/srcbucket23.txt' INTO TABLE t2 partition(ds2='part2') +PREHOOK: type: LOAD +PREHOOK: Output: default@t2@ds2=part2 +POSTHOOK: query: load data local inpath '../data/files/srcbucket23.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/srcbucket22.txt' INTO TABLE t3 partition(ds3='part1') +PREHOOK: type: LOAD +PREHOOK: Output: default@t3@ds3=part1 +POSTHOOK: query: load data local inpath '../data/files/srcbucket22.txt' INTO TABLE t3 partition(ds3='part1') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t3@ds3=part1 +PREHOOK: query: CREATE TABLE t4 (key4 int, value4 string) partitioned by (ds4 string) CLUSTERED BY (key4) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE t4 (key4 int, value4 string) partitioned by (ds4 string) CLUSTERED BY (key4) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@t4 +PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t4 partition(ds4='part1') +PREHOOK: type: LOAD +PREHOOK: Output: default@t4 +POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE t4 partition(ds4='part1') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t4 +POSTHOOK: Output: default@t4@ds4=part1 +PREHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t4 partition(ds4='part1') +PREHOOK: type: LOAD +PREHOOK: Output: default@t4@ds4=part1 +POSTHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE t4 partition(ds4='part1') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t4@ds4=part1 +PREHOOK: query: load data local inpath '../data/files/srcbucket22.txt' INTO TABLE t4 partition(ds4='part1') +PREHOOK: type: LOAD +PREHOOK: Output: default@t4@ds4=part1 +POSTHOOK: query: load data local inpath '../data/files/srcbucket22.txt' INTO TABLE t4 partition(ds4='part1') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t4@ds4=part1 +PREHOOK: query: load data local inpath '../data/files/srcbucket23.txt' INTO TABLE t4 partition(ds4='part1') +PREHOOK: type: LOAD +PREHOOK: Output: default@t4@ds4=part1 +POSTHOOK: query: load data local inpath '../data/files/srcbucket23.txt' INTO TABLE t4 partition(ds4='part1') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t4@ds4=part1 +PREHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 and a.ds1=b.ds2 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 and a.ds1=b.ds2 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (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_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) + +STAGE DEPENDENCIES: + Stage-4 is a root stage + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-4 + 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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1], Column[ds1]] + 1 [Column[key2], Column[ds2]] + Position of Big Table: 0 + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds1=part1/srcbucket20.txt=[ds2=part1/srcbucket20.txt], ds1=part1/srcbucket21.txt=[ds2=part1/srcbucket21.txt], ds1=part2/srcbucket22.txt=[ds2=part2/srcbucket22.txt], ds1=part2/srcbucket23.txt=[ds2=part2/srcbucket23.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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1], Column[ds1]] + 1 [Column[key2], Column[ds2]] + 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 5812 +#### 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 3062 +#### 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 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.t1 + name: default.t1 + + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 and a.ds1=b.ds2 +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 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 and a.ds1=b.ds2 +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 +#### A masked pattern was here #### +1028 +PREHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1) a) (TOK_TABREF (TOK_TABNAME t2) b) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key2)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) + +STAGE DEPENDENCIES: + Stage-4 is a root stage + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-4 + 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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1]] + 1 [Column[key2]] + 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/srcbucket22.txt], ds1=part1/srcbucket21.txt=[ds2=part1/srcbucket21.txt, ds2=part2/srcbucket23.txt], ds1=part2/srcbucket22.txt=[ds2=part1/srcbucket20.txt, ds2=part2/srcbucket22.txt], ds1=part2/srcbucket23.txt=[ds2=part1/srcbucket21.txt, ds2=part2/srcbucket23.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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1]] + 1 [Column[key2]] + 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 5812 +#### 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 3062 +#### 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 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.t1 + name: default.t1 + + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 +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 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t2 b +on a.key1=b.key2 +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 +#### A masked pattern was here #### +1028 +PREHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 and a.ds1=b.ds3 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 and a.ds1=b.ds3 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1) a) (TOK_TABREF (TOK_TABNAME t3) b) (and (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key3)) (= (. (TOK_TABLE_OR_COL a) ds1) (. (TOK_TABLE_OR_COL b) ds3))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) + +STAGE DEPENDENCIES: + Stage-4 is a root stage + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-4 + 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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1], Column[ds1]] + 1 [Column[key3], Column[ds3]] + Position of Big Table: 0 + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds1=part1/srcbucket20.txt=[ds3=part1/srcbucket20.txt], ds1=part1/srcbucket21.txt=[ds3=part1/srcbucket22.txt], ds1=part2/srcbucket22.txt=[], ds1=part2/srcbucket23.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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1], Column[ds1]] + 1 [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 5812 +#### 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 3062 +#### 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 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.t1 + name: default.t1 + + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 and a.ds1=b.ds3 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1@ds1=part1 +PREHOOK: Input: default@t1@ds1=part2 +PREHOOK: Input: default@t3@ds3=part1 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 and a.ds1=b.ds3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1@ds1=part1 +POSTHOOK: Input: default@t1@ds1=part2 +POSTHOOK: Input: default@t3@ds3=part1 +#### A masked pattern was here #### +220 +PREHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1) a) (TOK_TABREF (TOK_TABNAME t3) b) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key3)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) + +STAGE DEPENDENCIES: + Stage-4 is a root stage + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-4 + 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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1]] + 1 [Column[key3]] + Position of Big Table: 0 + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds1=part1/srcbucket20.txt=[ds3=part1/srcbucket20.txt], ds1=part1/srcbucket21.txt=[ds3=part1/srcbucket22.txt], ds1=part2/srcbucket22.txt=[ds3=part1/srcbucket20.txt], ds1=part2/srcbucket23.txt=[ds3=part1/srcbucket22.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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1]] + 1 [Column[key3]] + 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 5812 +#### 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 3062 +#### 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 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.t1 + name: default.t1 + + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1@ds1=part1 +PREHOOK: Input: default@t1@ds1=part2 +PREHOOK: Input: default@t3@ds3=part1 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t3 b +on a.key1=b.key3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1@ds1=part1 +POSTHOOK: Input: default@t1@ds1=part2 +POSTHOOK: Input: default@t3@ds3=part1 +#### A masked pattern was here #### +220 +PREHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 and a.ds1=b.ds4 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 and a.ds1=b.ds4 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1) a) (TOK_TABREF (TOK_TABNAME t4) b) (and (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key4)) (= (. (TOK_TABLE_OR_COL a) ds1) (. (TOK_TABLE_OR_COL b) ds4))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) + +STAGE DEPENDENCIES: + Stage-4 is a root stage + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-4 + 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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1], Column[ds1]] + 1 [Column[key4], Column[ds4]] + Position of Big Table: 0 + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds1=part1/srcbucket20.txt=[ds4=part1/srcbucket20.txt, ds4=part1/srcbucket22.txt], ds1=part1/srcbucket21.txt=[ds4=part1/srcbucket21.txt, ds4=part1/srcbucket23.txt], ds1=part2/srcbucket22.txt=[], ds1=part2/srcbucket23.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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1], Column[ds1]] + 1 [Column[key4], Column[ds4]] + 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 5812 +#### 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 3062 +#### 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 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.t1 + name: default.t1 + + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 and a.ds1=b.ds4 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1@ds1=part1 +PREHOOK: Input: default@t1@ds1=part2 +PREHOOK: Input: default@t4@ds4=part1 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 and a.ds1=b.ds4 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1@ds1=part1 +POSTHOOK: Input: default@t1@ds1=part2 +POSTHOOK: Input: default@t4@ds4=part1 +#### A masked pattern was here #### +464 +PREHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1) a) (TOK_TABREF (TOK_TABNAME t4) b) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key4)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) + +STAGE DEPENDENCIES: + Stage-4 is a root stage + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-4 + 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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1]] + 1 [Column[key4]] + Position of Big Table: 0 + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds1=part1/srcbucket20.txt=[ds4=part1/srcbucket20.txt, ds4=part1/srcbucket22.txt], ds1=part1/srcbucket21.txt=[ds4=part1/srcbucket21.txt, ds4=part1/srcbucket23.txt], ds1=part2/srcbucket22.txt=[ds4=part1/srcbucket20.txt, ds4=part1/srcbucket22.txt], ds1=part2/srcbucket23.txt=[ds4=part1/srcbucket21.txt, ds4=part1/srcbucket23.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 + 1 + handleSkewJoin: false + keys: + 0 [Column[key1]] + 1 [Column[key4]] + 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 5812 +#### 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 3062 +#### 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 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.t1 + name: default.t1 + + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1@ds1=part1 +PREHOOK: Input: default@t1@ds1=part2 +PREHOOK: Input: default@t4@ds4=part1 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ count(*) +from t1 a join t4 b +on a.key1=b.key4 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1@ds1=part1 +POSTHOOK: Input: default@t1@ds1=part2 +POSTHOOK: Input: default@t4@ds4=part1 +#### A masked pattern was here #### +1028