diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java index 6d26074..fa8ac8a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java @@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; @@ -114,6 +115,16 @@ private boolean checkBucketColumns(List bucketColumns, return keys.containsAll(bucketColumns); } + private boolean checkSortedColumns(List bucketCols, Table tbl) { + ListsortCols = tbl.getSortCols(); + List colName = new ArrayList(); + for (Order col : sortCols) { + colName.add(col.getCol()); + } + + return colName.containsAll(bucketCols); + } + private boolean checkBucketNumberAgainstBigTable( Map> aliasToBucketNumber, int bucketNumberInPart) { for (List bucketNums : aliasToBucketNumber.values()) { @@ -176,7 +187,8 @@ protected boolean canConvertMapJoinToBucketMapJoin( joinCtx, keysMap, baseBigAlias, - joinAliases); + joinAliases, + false); } /* @@ -193,7 +205,8 @@ protected boolean checkConvertBucketMapJoin( QBJoinTree joinCtx, Map> keysMap, String baseBigAlias, - List joinAliases) throws SemanticException { + List joinAliases, + boolean checkSorted) throws SemanticException { LinkedHashMap> aliasToPartitionBucketNumberMapping = new LinkedHashMap>(); @@ -292,6 +305,9 @@ protected boolean checkConvertBucketMapJoin( if (!checkBucketColumns(p.getBucketCols(), keys, orders)) { return false; } + if (checkSorted && !checkSortedColumns(p.getBucketCols(), tbl)) { + return false; + } List fileNames = getOnePartitionBucketFileNames(p.getDataLocation(), pGraphContext); // The number of files for the table should be same as number of buckets. @@ -322,6 +338,9 @@ protected boolean checkConvertBucketMapJoin( if (!checkBucketColumns(tbl.getBucketCols(), keys, orders)) { return false; } + if (checkSorted && !checkSortedColumns(tbl.getBucketCols(), tbl)) { + return false; + } List fileNames = getOnePartitionBucketFileNames(tbl.getDataLocation(), pGraphContext); Integer num = new Integer(tbl.getNumBuckets()); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java index f079895..5895d13 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java @@ -501,7 +501,8 @@ protected boolean canConvertJoinToBucketMapJoin( joinCtx, keyExprMap, joinAlias, - Arrays.asList(srcs)); + Arrays.asList(srcs), + true); } // Convert the join operator to a bucket map-join join operator