diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java index f064efd..75cdf4f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java @@ -25,8 +25,8 @@ import org.apache.commons.logging.Log; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.io.IOContext; +import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork.BucketMapJoinContext; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.util.ReflectionUtils; @@ -43,7 +43,7 @@ public class ExecMapperContext { // so it won't be updated. private String currentInputFile = null; private boolean inputFileChecked = false; - private Integer fileId = new Integer(-1); + private String fileId; private MapredLocalWork localWork = null; private Map fetchOperators; private JobConf jc; @@ -147,11 +147,11 @@ public class ExecMapperContext { this.localWork = localWork; } - public Integer getFileId() { + public String getFileId() { return fileId; } - public void setFileId(Integer fileId) { + public void setFileId(String fileId) { this.fileId = fileId; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index dc3d366..d94cd82 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -398,7 +398,7 @@ public class FileSinkOperator extends TerminalOperator implements int filesIdx = 0; Set seenBuckets = new HashSet(); for (int idx = 0; idx < totalFiles; idx++) { - if (this.getExecContext() != null && this.getExecContext().getFileId() != -1) { + if (this.getExecContext() != null && this.getExecContext().getFileId() != null) { LOG.info("replace taskId from execContext "); taskId = Utilities.replaceTaskIdFromFilename(taskId, this.getExecContext().getFileId()); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java index 5750599..13884cd 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java @@ -399,12 +399,13 @@ public class HashTableSinkOperator extends TerminalOperator i HashMapWrapper hashTable = hashTables.getValue(); // get current input file name - String bigBucketFileName = this.getExecContext().getCurrentBigBucketFile(); - if (bigBucketFileName == null || bigBucketFileName.length() == 0) { - bigBucketFileName = "-"; - } + String bigBucketFileName = getExecContext().getCurrentBigBucketFile(); + + String fileName = getExecContext().getLocalWork().getBucketFileName(bigBucketFileName); + // get the tmp URI path; it will be a hdfs path if not local mode - String tmpURIPath = Utilities.generatePath(tmpURI, conf.getDumpFilePrefix(), tag, bigBucketFileName); + String dumpFilePrefix = conf.getDumpFilePrefix(); + String tmpURIPath = Utilities.generatePath(tmpURI, dumpFilePrefix, tag, fileName); hashTable.isAbort(rowNumber, console); console.printInfo(Utilities.now() + "\tDump the hashtable into file: " + tmpURIPath); // get the hashtable file and path diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java index 8f2804f..9df7acd 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java @@ -154,16 +154,10 @@ public class MapJoinOperator extends AbstractMapJoinOperator implem boolean localMode = HiveConf.getVar(hconf, HiveConf.ConfVars.HADOOPJT).equals("local"); String baseDir = null; - String currentInputFile = HiveConf.getVar(hconf, HiveConf.ConfVars.HADOOPMAPFILENAME); + String currentInputFile = getExecContext().getCurrentInputFile(); LOG.info("******* Load from HashTable File: input : " + currentInputFile); - String currentFileName; - - if (this.getExecContext().getLocalWork().getInputFileChangeSensitive()) { - currentFileName = this.getFileName(currentInputFile); - } else { - currentFileName = "-"; - } + String fileName = getExecContext().getLocalWork().getBucketFileName(currentInputFile); try { if (localMode) { @@ -188,7 +182,7 @@ public class MapJoinOperator extends AbstractMapJoinOperator implem .entrySet()) { Byte pos = entry.getKey(); HashMapWrapper hashtable = entry.getValue(); - String filePath = Utilities.generatePath(baseDir, conf.getDumpFilePrefix(), pos, currentFileName); + String filePath = Utilities.generatePath(baseDir, conf.getDumpFilePrefix(), pos, fileName); Path path = new Path(filePath); LOG.info("\tLoad back 1 hashtable file from tmp file uri:" + path.toString()); hashtable.initilizePersistentHash(path.toUri().getPath()); @@ -283,17 +277,6 @@ public class MapJoinOperator extends AbstractMapJoinOperator implem } } - private String getFileName(String path) { - if (path == null || path.length() == 0) { - return null; - } - - int last_separator = path.lastIndexOf(Path.SEPARATOR) + 1; - String fileName = path.substring(last_separator); - return fileName; - - } - @Override public void closeOp(boolean abort) throws HiveException { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java index c1e7e5e..9154eec 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java @@ -50,9 +50,9 @@ import org.apache.hadoop.hive.ql.exec.persistence.AbstractMapJoinKey; import org.apache.hadoop.hive.ql.exec.persistence.HashMapWrapper; import org.apache.hadoop.hive.ql.exec.persistence.MapJoinObjectValue; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork.BucketMapJoinContext; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; @@ -319,8 +319,7 @@ public class MapredLocalTask extends Task implements Serializab InspectableObject row = fetchOp.getNextRow(); if (row == null) { if (inputFileChangeSenstive) { - String fileName = this.getFileName(bigTableBucket); - execContext.setCurrentBigBucketFile(fileName); + execContext.setCurrentBigBucketFile(bigTableBucket); forwardOp.reset(); } forwardOp.close(false); @@ -410,12 +409,11 @@ public class MapredLocalTask extends Task implements Serializab HashMapWrapper hashTable = new HashMapWrapper(); - if (bigBucketFileName == null || bigBucketFileName.length() == 0) { - bigBucketFileName = "-"; - } + String fileName = work.getBucketFileName(bigBucketFileName); + HashTableSinkOperator htso = (HashTableSinkOperator)childOp; String tmpURIPath = Utilities.generatePath(tmpURI, htso.getConf().getDumpFilePrefix(), - tag, bigBucketFileName); + tag, fileName); console.printInfo(Utilities.now() + "\tDump the hashtable into file: " + tmpURIPath); Path path = new Path(tmpURIPath); FileSystem fs = path.getFileSystem(job); @@ -443,17 +441,6 @@ public class MapredLocalTask extends Task implements Serializab fetchOp.setupContext(iter, null); } - private String getFileName(String path) { - if (path == null || path.length() == 0) { - return null; - } - - int last_separator = path.lastIndexOf(Path.SEPARATOR) + 1; - String fileName = path.substring(last_separator); - return fileName; - - } - @Override public void localizeMRTmpFilesImpl(Context ctx) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java index b830ad9..09cf9a6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java @@ -31,11 +31,11 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.persistence.RowContainer; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork.BucketMapJoinContext; import org.apache.hadoop.hive.ql.plan.api.OperatorType; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject; @@ -482,7 +482,7 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp .getBucketMatcherClass(); BucketMatcher bucketMatcher = (BucketMatcher) ReflectionUtils.newInstance( bucketMatcherCls, null); - this.getExecContext().setFileId(bucketMatcherCxt.getBucketFileNameMapping().get(currentInputFile)); + this.getExecContext().setFileId(bucketMatcherCxt.prependedBucketID(currentInputFile)); LOG.info("set task id: " + this.getExecContext().getFileId()); bucketMatcher.setAliasBucketFileNameMapping(bucketMatcherCxt diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index e15e265..c16331f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -1146,6 +1146,12 @@ public final class Utilities { private static Pattern fileNameTaskIdRegex = Pattern.compile("^.*?([0-9]+)(_[0-9]{1,3})?(\\..*)?$"); /** + * This retruns prefix part + taskID for bucket join for partitioned table + */ + private static Pattern fileNamePrefixedTaskIdRegex = + Pattern.compile("^.*?((\\(.*\\))?[0-9]+)(_[0-9]{1,3})?(\\..*)?$"); + + /** * Get the task id from the filename. It is assumed that the filename is derived from the output * of getTaskId * @@ -1153,13 +1159,28 @@ public final class Utilities { * filename to extract taskid from */ public static String getTaskIdFromFilename(String filename) { + return getIdFromFilename(filename, fileNameTaskIdRegex); + } + + /** + * Get the part-spec + task id from the filename. It is assumed that the filename is derived + * from the output of getTaskId + * + * @param filename + * filename to extract taskid from + */ + public static String getPrefixedTaskIdFromFilename(String filename) { + return getIdFromFilename(filename, fileNamePrefixedTaskIdRegex); + } + + private static String getIdFromFilename(String filename, Pattern pattern) { String taskId = filename; int dirEnd = filename.lastIndexOf(Path.SEPARATOR); if (dirEnd != -1) { taskId = filename.substring(dirEnd + 1); } - Matcher m = fileNameTaskIdRegex.matcher(taskId); + Matcher m = pattern.matcher(taskId); if (!m.matches()) { LOG.warn("Unable to get task id from file name: " + filename + ". Using last component" + taskId + " as task id."); @@ -1178,14 +1199,21 @@ public final class Utilities { * filename to replace taskid "0_0" or "0_0.gz" by 33 to "33_0" or "33_0.gz" */ public static String replaceTaskIdFromFilename(String filename, int bucketNum) { + return replaceTaskIdFromFilename(filename, String.valueOf(bucketNum)); + } + + public static String replaceTaskIdFromFilename(String filename, String strBucketNum) { String taskId = getTaskIdFromFilename(filename); - String newTaskId = replaceTaskId(taskId, bucketNum); + String newTaskId = replaceTaskId(taskId, strBucketNum); String ret = replaceTaskIdFromFilename(filename, taskId, newTaskId); return (ret); } private static String replaceTaskId(String taskId, int bucketNum) { - String strBucketNum = String.valueOf(bucketNum); + return replaceTaskId(taskId, String.valueOf(bucketNum)); + } + + private static String replaceTaskId(String taskId, String strBucketNum) { int bucketNumLen = strBucketNum.length(); int taskIdLen = taskId.length(); StringBuffer s = new StringBuffer(); @@ -1411,7 +1439,7 @@ public final class Utilities { throw new IOException("Unable to delete tmp file: " + one.getPath()); } } else { - String taskId = getTaskIdFromFilename(one.getPath().getName()); + String taskId = getPrefixedTaskIdFromFilename(one.getPath().getName()); FileStatus otherFile = taskIdToFile.get(taskId); if (otherFile == null) { taskIdToFile.put(taskId, one); 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 4811b78..8fe8aac 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java @@ -22,6 +22,7 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; @@ -373,9 +374,21 @@ public class BucketMapJoinOptimizer implements Transform { } desc.setAliasBucketFileNameMapping(aliasBucketFileNameMapping); desc.setBigTableAlias(baseBigAlias); + if (bigTblPartsToBucketFileNames.size() > 1) { + desc.setPartSpecToFileMapping(convert(bigTblPartsToBucketFileNames)); + } return null; } + // convert partition to partition spec string + private Map> convert(Map> mapping) { + Map> converted = new HashMap>(); + for (Map.Entry> entry : mapping.entrySet()) { + converted.put(entry.getKey().getName(), entry.getValue()); + } + return converted; + } + private void fillMapping(String baseBigAlias, LinkedHashMap aliasToBucketNumberMapping, LinkedHashMap> aliasToBucketFileNamesMapping, diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java index 4abbd1f..22f2073 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java @@ -58,6 +58,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; import org.apache.hadoop.hive.ql.parse.RowResolver; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; @@ -69,7 +70,6 @@ import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.hive.ql.plan.FilterDesc.sampleDesc; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork.BucketMapJoinContext; /** * General utility common functions for the Processor to convert operator into @@ -280,6 +280,7 @@ public final class GenMapRedUtils { localPlan.setInputFileChangeSensitive(true); bucketMJCxt.setMapJoinBigTableAlias(currMapJoinOp.getConf().getBigTableAlias()); bucketMJCxt.setBucketMatcherClass(org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class); + bucketMJCxt.setPartSpecToFileMapping(currMapJoinOp.getConf().getPartSpecToFileMapping()); } } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java index 0923562..d2a7707 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java @@ -129,6 +129,7 @@ public class MapJoinResolver implements PhysicalPlanResolver { newLocalWork.setDummyParentOp(dummyOps); newLocalWork.setTmpFileURI(tmpFileURI); newLocalWork.setInputFileChangeSensitive(localwork.getInputFileChangeSensitive()); + newLocalWork.setBucketMapjoinContext(localwork.copyPartSpecMapping()); mapredWork.setMapLocalWork(newLocalWork); // get all parent tasks List> parentTasks = currTask.getParentTasks(); diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/BucketMapJoinContext.java ql/src/java/org/apache/hadoop/hive/ql/plan/BucketMapJoinContext.java new file mode 100644 index 0000000..c83d6ee --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/BucketMapJoinContext.java @@ -0,0 +1,183 @@ +package org.apache.hadoop.hive.ql.plan; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.BucketMatcher; + +import java.io.File; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +public class BucketMapJoinContext implements Serializable { + + private static final long serialVersionUID = 1L; + + // used for bucket map join + private LinkedHashMap>> aliasBucketFileNameMapping; + private String mapJoinBigTableAlias; + private Class bucketMatcherClass; + + private LinkedHashMap>> aliasBucketBaseFileNameMapping; + private LinkedHashMap bucketFileNameMapping; + private Map> partSpecToFileMapping; + + private transient Map inputToPartSpecMapping; + + public BucketMapJoinContext() {} + + public BucketMapJoinContext(MapJoinDesc clone) { + this.mapJoinBigTableAlias = clone.getBigTableAlias(); + this.aliasBucketFileNameMapping = clone.getAliasBucketFileNameMapping(); + this.bucketFileNameMapping = clone.getBucketFileNameMapping(); + this.partSpecToFileMapping = clone.getPartSpecToFileMapping(); + } + + public void setMapJoinBigTableAlias(String bigTableAlias) { + this.mapJoinBigTableAlias = bigTableAlias; + } + + + public void deriveBucketMapJoinMapping() { + if (aliasBucketFileNameMapping != null) { + Iterator>>> iter = + aliasBucketFileNameMapping.entrySet().iterator(); + aliasBucketBaseFileNameMapping = new LinkedHashMap>>(); + + while (iter.hasNext()) { + Map.Entry>> old = iter.next(); + + LinkedHashMap> newBucketBaseFileNameMapping = new LinkedHashMap>(); + Iterator>> oldAliasFileNameMappingIter = old.getValue().entrySet().iterator(); + while (oldAliasFileNameMappingIter.hasNext()) { + //For a give table and its bucket full file path list, only keep the base file name (remove file path etc). + //And put the new list into the new mapping. + Map.Entry> oldTableBucketFileNames = oldAliasFileNameMappingIter.next(); + ArrayList oldTableBucketNames = oldTableBucketFileNames.getValue(); + ArrayList newTableBucketFileBaseName = new ArrayList(oldTableBucketNames.size()); + //for each bucket file, only keep its base files and store into a new list. + if (oldTableBucketNames != null) { + for (String bucketFName : oldTableBucketNames) { + newTableBucketFileBaseName.add(getBaseFileName(bucketFName)); + } + } + String bigTblBucketFileName = getBaseFileName(oldTableBucketFileNames.getKey()); + if (newBucketBaseFileNameMapping.containsKey(bigTblBucketFileName)) { + String fullPath = oldTableBucketFileNames.getKey(); + String dir = getBaseFileName(fullPath.substring(0, fullPath.lastIndexOf(bigTblBucketFileName))); + bigTblBucketFileName = dir + File.separator + bigTblBucketFileName; + } + //put the new mapping + newBucketBaseFileNameMapping.put(bigTblBucketFileName, newTableBucketFileBaseName); + } + String tableAlias = old.getKey(); + aliasBucketBaseFileNameMapping.put(tableAlias, newBucketBaseFileNameMapping); + } + } + } + + private String getBaseFileName(String path) { + try { + return ((new Path(path)).getName()); + } catch (Exception ex) { + // This could be due to either URI syntax error or File constructor + // illegal arg; we don't really care which one it is. + return path; + } + } + + public String getMapJoinBigTableAlias() { + return mapJoinBigTableAlias; + } + + public Class getBucketMatcherClass() { + return bucketMatcherClass; + } + + public void setBucketMatcherClass( + Class bucketMatcherClass) { + this.bucketMatcherClass = bucketMatcherClass; + } + + @Explain(displayName = "Alias Bucket File Name Mapping", normalExplain = false) + public LinkedHashMap>> getAliasBucketFileNameMapping() { + return aliasBucketFileNameMapping; + } + + public void setAliasBucketFileNameMapping( + LinkedHashMap>> aliasBucketFileNameMapping) { + this.aliasBucketFileNameMapping = aliasBucketFileNameMapping; + } + + @Override + public String toString() { + if (aliasBucketFileNameMapping != null) { + return "Mapping:" + aliasBucketFileNameMapping.toString(); + } else { + return ""; + } + } + + @Explain(displayName = "Alias Bucket Base File Name Mapping", normalExplain = false) + public LinkedHashMap>> getAliasBucketBaseFileNameMapping() { + return aliasBucketBaseFileNameMapping; + } + + public void setAliasBucketBaseFileNameMapping( + LinkedHashMap>> aliasBucketBaseFileNameMapping) { + this.aliasBucketBaseFileNameMapping = aliasBucketBaseFileNameMapping; + } + + @Explain(displayName = "Alias Bucket Output File Name Mapping", normalExplain = false) + public LinkedHashMap getBucketFileNameMapping() { + return bucketFileNameMapping; + } + + public void setBucketFileNameMapping(LinkedHashMap bucketFileNameMapping) { + this.bucketFileNameMapping = bucketFileNameMapping; + } + + public Map> getPartSpecToFileMapping() { + return partSpecToFileMapping; + } + + public void setPartSpecToFileMapping(Map> partSpecToFileMapping) { + this.partSpecToFileMapping = partSpecToFileMapping; + } + + public String prependedBucketID(String input) { + String taskId = String.valueOf(bucketFileNameMapping.get(input)); + if (partSpecToFileMapping != null) { + return prefixPartSpec(input, taskId); + } + return taskId; + } + + public String prependBucketFile(String input, String name) { + if (partSpecToFileMapping != null) { + return prefixPartSpec(input, name); + } + return name; + } + + private String prefixPartSpec(String input, String name) { + Map mapping = inputToPartSpecMapping == null ? + inputToPartSpecMapping = convert(partSpecToFileMapping) : inputToPartSpecMapping; + String prefix = mapping.get(input); + return prefix == null || prefix.isEmpty() ? name : "(" + prefix + ")" + name; + } + + private Map convert(Map> mapping) { + Map converted = new HashMap(); + for (Map.Entry> entry : mapping.entrySet()) { + String partSpec = entry.getKey(); + for (String file : entry.getValue()) { + converted.put(file, partSpec); + } + } + return converted; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java index 947ca90..d8e233f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java @@ -22,7 +22,6 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -75,17 +74,14 @@ public class HashTableSinkDesc extends JoinDesc implements Serializable { private Map> retainList; - private transient String bigTableAlias; - - private LinkedHashMap>> aliasBucketFileNameMapping; - private LinkedHashMap bucketFileNameMapping; + private transient BucketMapJoinContext bucketMapjoinContext; private float hashtableMemoryUsage; //map join dump file name private String dumpFilePrefix; public HashTableSinkDesc() { - bucketFileNameMapping = new LinkedHashMap(); + bucketMapjoinContext = new BucketMapJoinContext(); } public HashTableSinkDesc(MapJoinDesc clone) { @@ -109,10 +105,8 @@ public class HashTableSinkDesc extends JoinDesc implements Serializable { this.valueTblFilteredDescs = clone.getValueFilteredTblDescs(); this.posBigTable = clone.getPosBigTable(); this.retainList = clone.getRetainList(); - this.bigTableAlias = clone.getBigTableAlias(); - this.aliasBucketFileNameMapping = clone.getAliasBucketFileNameMapping(); - this.bucketFileNameMapping = clone.getBucketFileNameMapping(); this.dumpFilePrefix = clone.getDumpFilePrefix(); + this.bucketMapjoinContext = new BucketMapJoinContext(clone); } @@ -362,34 +356,11 @@ public class HashTableSinkDesc extends JoinDesc implements Serializable { this.valueTblDescs = valueTblDescs; } - /** - * @return bigTableAlias - */ - public String getBigTableAlias() { - return bigTableAlias; - } - - /** - * @param bigTableAlias - */ - public void setBigTableAlias(String bigTableAlias) { - this.bigTableAlias = bigTableAlias; - } - - public LinkedHashMap>> getAliasBucketFileNameMapping() { - return aliasBucketFileNameMapping; - } - - public void setAliasBucketFileNameMapping( - LinkedHashMap>> aliasBucketFileNameMapping) { - this.aliasBucketFileNameMapping = aliasBucketFileNameMapping; - } - - public LinkedHashMap getBucketFileNameMapping() { - return bucketFileNameMapping; + public BucketMapJoinContext getBucketMapjoinContext() { + return bucketMapjoinContext; } - public void setBucketFileNameMapping(LinkedHashMap bucketFileNameMapping) { - this.bucketFileNameMapping = bucketFileNameMapping; + public void setBucketMapjoinContext(BucketMapJoinContext bucketMapjoinContext) { + this.bucketMapjoinContext = bucketMapjoinContext; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java index 6d0a6a5..c1a30af 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java @@ -49,6 +49,7 @@ public class MapJoinDesc extends JoinDesc implements Serializable { private LinkedHashMap>> aliasBucketFileNameMapping; private LinkedHashMap bucketFileNameMapping; + private Map> partSpecToFileMapping; //map join dump file name private String dumpFilePrefix; @@ -67,6 +68,7 @@ public class MapJoinDesc extends JoinDesc implements Serializable { this.bigTableAlias = clone.bigTableAlias; this.aliasBucketFileNameMapping = clone.aliasBucketFileNameMapping; this.bucketFileNameMapping = clone.bucketFileNameMapping; + this.partSpecToFileMapping = clone.partSpecToFileMapping; this.dumpFilePrefix = clone.dumpFilePrefix; } @@ -223,4 +225,12 @@ public class MapJoinDesc extends JoinDesc implements Serializable { public void setBucketFileNameMapping(LinkedHashMap bucketFileNameMapping) { this.bucketFileNameMapping = bucketFileNameMapping; } + + public Map> getPartSpecToFileMapping() { + return partSpecToFileMapping; + } + + public void setPartSpecToFileMapping(Map> partToFileMapping) { + this.partSpecToFileMapping = partToFileMapping; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java index 319e356..63ca8d4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java @@ -18,16 +18,11 @@ package org.apache.hadoop.hive.ql.plan; -import java.io.File; import java.io.Serializable; -import java.util.ArrayList; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; -import java.util.Map.Entry; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.BucketMatcher; import org.apache.hadoop.hive.ql.exec.Operator; /** @@ -143,6 +138,15 @@ public class MapredLocalWork implements Serializable { this.bucketMapjoinContext = bucketMapjoinContext; } + public BucketMapJoinContext copyPartSpecMapping() { + if (bucketMapjoinContext != null && bucketMapjoinContext.getPartSpecToFileMapping() != null) { + BucketMapJoinContext context = new BucketMapJoinContext(); + context.setPartSpecToFileMapping(bucketMapjoinContext.getPartSpecToFileMapping()); + return context; + } + return null; + } + public void setTmpFileURI(String tmpFileURI) { this.tmpFileURI = tmpFileURI; } @@ -151,121 +155,22 @@ public class MapredLocalWork implements Serializable { return tmpFileURI; } - public static class BucketMapJoinContext implements Serializable { - - private static final long serialVersionUID = 1L; - - // used for bucket map join - private LinkedHashMap>> aliasBucketFileNameMapping; - private String mapJoinBigTableAlias; - private Class bucketMatcherClass; - - private LinkedHashMap>> aliasBucketBaseFileNameMapping; - private LinkedHashMap bucketFileNameMapping; - - public void setMapJoinBigTableAlias(String bigTableAlias) { - this.mapJoinBigTableAlias = bigTableAlias; - } - - - public void deriveBucketMapJoinMapping() { - if (aliasBucketFileNameMapping != null) { - Iterator>>> iter = - aliasBucketFileNameMapping.entrySet().iterator(); - aliasBucketBaseFileNameMapping = new LinkedHashMap>>(); - - while (iter.hasNext()) { - Entry>> old = iter.next(); - - LinkedHashMap> newBucketBaseFileNameMapping = new LinkedHashMap>(); - Iterator>> oldAliasFileNameMappingIter = old.getValue().entrySet().iterator(); - while (oldAliasFileNameMappingIter.hasNext()) { - //For a give table and its bucket full file path list, only keep the base file name (remove file path etc). - //And put the new list into the new mapping. - Entry> oldTableBucketFileNames = oldAliasFileNameMappingIter.next(); - ArrayList oldTableBucketNames = oldTableBucketFileNames.getValue(); - ArrayList newTableBucketFileBaseName = new ArrayList (oldTableBucketNames.size()); - //for each bucket file, only keep its base files and store into a new list. - if (oldTableBucketNames != null) { - for (String bucketFName : oldTableBucketNames) { - newTableBucketFileBaseName.add(getBaseFileName(bucketFName)); - } - } - String bigTblBucketFileName = getBaseFileName(oldTableBucketFileNames.getKey()); - if(newBucketBaseFileNameMapping.containsKey(bigTblBucketFileName)) { - String fullPath = oldTableBucketFileNames.getKey(); - String dir = getBaseFileName(fullPath.substring(0, fullPath.lastIndexOf(bigTblBucketFileName))); - bigTblBucketFileName = dir + File.separator + bigTblBucketFileName; - } - //put the new mapping - newBucketBaseFileNameMapping.put(bigTblBucketFileName, newTableBucketFileBaseName); - } - String tableAlias = old.getKey(); - aliasBucketBaseFileNameMapping.put(tableAlias, newBucketBaseFileNameMapping); - } - } - } - - private String getBaseFileName (String path) { - try { - return ((new Path(path)).getName()); - } catch (Exception ex) { - // This could be due to either URI syntax error or File constructor - // illegal arg; we don't really care which one it is. - return path; - } - } - - public String getMapJoinBigTableAlias() { - return mapJoinBigTableAlias; - } - - public Class getBucketMatcherClass() { - return bucketMatcherClass; + public String getBucketFileName(String bigFileName) { + if (!inputFileChangeSensitive || bigFileName == null || bigFileName.length() == 0) { + return "-"; } - - public void setBucketMatcherClass( - Class bucketMatcherClass) { - this.bucketMatcherClass = bucketMatcherClass; - } - - @Explain(displayName = "Alias Bucket File Name Mapping", normalExplain = false) - public LinkedHashMap>> getAliasBucketFileNameMapping() { - return aliasBucketFileNameMapping; - } - - public void setAliasBucketFileNameMapping( - LinkedHashMap>> aliasBucketFileNameMapping) { - this.aliasBucketFileNameMapping = aliasBucketFileNameMapping; - } - - @Override - public String toString() { - if (aliasBucketFileNameMapping != null) { - return "Mapping:" + aliasBucketFileNameMapping.toString(); - } else { - return ""; - } - } - - @Explain(displayName = "Alias Bucket Base File Name Mapping", normalExplain = false) - public LinkedHashMap>> getAliasBucketBaseFileNameMapping() { - return aliasBucketBaseFileNameMapping; - } - - public void setAliasBucketBaseFileNameMapping( - LinkedHashMap>> aliasBucketBaseFileNameMapping) { - this.aliasBucketBaseFileNameMapping = aliasBucketBaseFileNameMapping; - } - - @Explain(displayName = "Alias Bucket Output File Name Mapping", normalExplain = false) - public LinkedHashMap getBucketFileNameMapping() { - return bucketFileNameMapping; + String fileName = getFileName(bigFileName); + if (bucketMapjoinContext != null) { + fileName = bucketMapjoinContext.prependBucketFile(bigFileName, fileName); } + return fileName; + } - public void setBucketFileNameMapping(LinkedHashMap bucketFileNameMapping) { - this.bucketFileNameMapping = bucketFileNameMapping; + private String getFileName(String path) { + int last_separator = path.lastIndexOf(Path.SEPARATOR); + if (last_separator < 0) { + return path; } - + return path.substring(last_separator + 1); } } diff --git ql/src/test/queries/clientpositive/bucketmapjoin_multipart.q ql/src/test/queries/clientpositive/bucketmapjoin_multipart.q new file mode 100644 index 0000000..1519c05 --- /dev/null +++ ql/src/test/queries/clientpositive/bucketmapjoin_multipart.q @@ -0,0 +1,62 @@ +create table hive_test_smb_bucket1 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 2 buckets; +create table hive_test_smb_bucket2 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 4 buckets; + +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; + +insert overwrite table hive_test_smb_bucket1 partition (ds='2010-10-15') select key, value from src; +insert overwrite table hive_test_smb_bucket2 partition (ds='2010-10-15') select key, value from src; + +select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; +select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; +select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; +select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +insert overwrite table hive_test_smb_bucket1 partition (ds='2012-07-02') select key, value from src1; + +set hive.optimize.bucketmapjoin = false; +set hive.optimize.bucketmapjoin.sortedmerge = false; +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; +select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; +select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; +select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +insert overwrite table hive_test_smb_bucket2 partition (ds='2012-07-02') select key, value from src1; + +set hive.optimize.bucketmapjoin = false; +set hive.optimize.bucketmapjoin.sortedmerge = false; +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; +select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; +select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key; + +-- Not yet for SMBJoin + +drop table hive_test_smb_bucket1; +drop table hive_test_smb_bucket2; \ No newline at end of file diff --git ql/src/test/results/clientpositive/bucketmapjoin5.q.out ql/src/test/results/clientpositive/bucketmapjoin5.q.out index 0b408ec..b0d0424 100644 --- ql/src/test/results/clientpositive/bucketmapjoin5.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin5.q.out @@ -232,6 +232,7 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Bucket Mapjoin Context: Needs Tagging: false Path -> Alias: #### A masked pattern was here #### @@ -689,6 +690,7 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Bucket Mapjoin Context: Needs Tagging: false Path -> Alias: #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/bucketmapjoin_multipart.q.out ql/src/test/results/clientpositive/bucketmapjoin_multipart.q.out new file mode 100644 index 0000000..9d42146 --- /dev/null +++ ql/src/test/results/clientpositive/bucketmapjoin_multipart.q.out @@ -0,0 +1,2150 @@ +PREHOOK: query: create table hive_test_smb_bucket1 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 2 buckets +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table hive_test_smb_bucket1 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 2 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@hive_test_smb_bucket1 +PREHOOK: query: create table hive_test_smb_bucket2 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 4 buckets +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table hive_test_smb_bucket2 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 4 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@hive_test_smb_bucket2 +PREHOOK: query: insert overwrite table hive_test_smb_bucket1 partition (ds='2010-10-15') select key, value from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: query: insert overwrite table hive_test_smb_bucket1 partition (ds='2010-10-15') select key, value from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: insert overwrite table hive_test_smb_bucket2 partition (ds='2010-10-15') select key, value from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@hive_test_smb_bucket2@ds=2010-10-15 +POSTHOOK: query: insert overwrite table hive_test_smb_bucket2 partition (ds='2010-10-15') select key, value from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@hive_test_smb_bucket2@ds=2010-10-15 +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +1028 +PREHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket1) a) (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (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[key]] + 1 [Column[key]] + Position of Big Table: 0 + + 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[key]] + 1 [Column[key]] + 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: ds=2010-10-15 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2010-10-15 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 1 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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 + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 1 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 + + 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 hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +1028 +PREHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket1) a) (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (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[key]] + 1 [Column[key]] + Position of Big Table: 0 + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {000000_0=[000000_0]} + 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[key]] + 1 [Column[key]] + 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: ds=2010-10-15 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2010-10-15 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 1 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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 + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 1 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 + + 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 hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +1028 +PREHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket1) a) (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (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-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + GatherStats: false + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 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 + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2010-10-15 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2010-10-15 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 1 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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 + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 1 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 + + 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 hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +1028 +PREHOOK: query: insert overwrite table hive_test_smb_bucket1 partition (ds='2012-07-02') select key, value from src1 +PREHOOK: type: QUERY +PREHOOK: Input: default@src1 +PREHOOK: Output: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: query: insert overwrite table hive_test_smb_bucket1 partition (ds='2012-07-02') select key, value from src1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src1 +POSTHOOK: Output: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +1065 +PREHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket1) a) (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (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[key]] + 1 [Column[key]] + Position of Big Table: 0 + + 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[key]] + 1 [Column[key]] + 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: ds=2010-10-15 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2010-10-15 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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 + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 +#### A masked pattern was here #### + Partition + base file name: ds=2012-07-02 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2012-07-02 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 25 + partition_columns ds + rawDataSize 211 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 236 +#### 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: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 + + 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 hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +1065 +PREHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket1) a) (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (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[key]] + 1 [Column[key]] + Position of Big Table: 0 + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {000000_0=[000000_0], ds=2012-07-02/000000_0=[000000_0]} + 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[key]] + 1 [Column[key]] + 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 + Bucket Mapjoin Context: + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2010-10-15 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2010-10-15 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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 + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 +#### A masked pattern was here #### + Partition + base file name: ds=2012-07-02 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2012-07-02 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 25 + partition_columns ds + rawDataSize 211 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 236 +#### 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: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 + + 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 hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +1065 +PREHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket1) a) (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (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-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + GatherStats: false + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 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 + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2010-10-15 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2010-10-15 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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 + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 +#### A masked pattern was here #### + Partition + base file name: ds=2012-07-02 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2012-07-02 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 25 + partition_columns ds + rawDataSize 211 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 236 +#### 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: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 + + 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 hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +1065 +PREHOOK: query: insert overwrite table hive_test_smb_bucket2 partition (ds='2012-07-02') select key, value from src1 +PREHOOK: type: QUERY +PREHOOK: Input: default@src1 +PREHOOK: Output: default@hive_test_smb_bucket2@ds=2012-07-02 +POSTHOOK: query: insert overwrite table hive_test_smb_bucket2 partition (ds='2012-07-02') select key, value from src1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src1 +POSTHOOK: Output: default@hive_test_smb_bucket2@ds=2012-07-02 +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2012-07-02 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2012-07-02 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +1117 +PREHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket1) a) (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (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[key]] + 1 [Column[key]] + Position of Big Table: 0 + + 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[key]] + 1 [Column[key]] + 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: ds=2010-10-15 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2010-10-15 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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 + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 +#### A masked pattern was here #### + Partition + base file name: ds=2012-07-02 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2012-07-02 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 25 + partition_columns ds + rawDataSize 211 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 236 +#### 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: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 + + 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 hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2012-07-02 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2012-07-02 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +1117 +PREHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket1) a) (TOK_TABREF (TOK_TABNAME hive_test_smb_bucket2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (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[key]] + 1 [Column[key]] + Position of Big Table: 0 + + 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[key]] + 1 [Column[key]] + 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: ds=2010-10-15 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2010-10-15 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 500 + partition_columns ds + rawDataSize 5312 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + 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 + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 +#### A masked pattern was here #### + Partition + base file name: ds=2012-07-02 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2012-07-02 + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 1 + numPartitions 2 + numRows 25 + partition_columns ds + rawDataSize 211 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 236 +#### 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: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.hive_test_smb_bucket1 + numFiles 2 + numPartitions 2 + numRows 525 + partition_columns ds + rawDataSize 5523 + serialization.ddl struct hive_test_smb_bucket1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 6048 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.hive_test_smb_bucket1 + name: default.hive_test_smb_bucket1 + + 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 hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +PREHOOK: Input: default@hive_test_smb_bucket2@ds=2012-07-02 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(b) */ count(*) FROM hive_test_smb_bucket1 a JOIN hive_test_smb_bucket2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket1@ds=2012-07-02 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 +POSTHOOK: Input: default@hive_test_smb_bucket2@ds=2012-07-02 +#### A masked pattern was here #### +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +1117 +PREHOOK: query: -- Not yet for SMBJoin + +drop table hive_test_smb_bucket1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@hive_test_smb_bucket1 +PREHOOK: Output: default@hive_test_smb_bucket1 +POSTHOOK: query: -- Not yet for SMBJoin + +drop table hive_test_smb_bucket1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@hive_test_smb_bucket1 +POSTHOOK: Output: default@hive_test_smb_bucket1 +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: drop table hive_test_smb_bucket2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@hive_test_smb_bucket2 +PREHOOK: Output: default@hive_test_smb_bucket2 +POSTHOOK: query: drop table hive_test_smb_bucket2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@hive_test_smb_bucket2 +POSTHOOK: Output: default@hive_test_smb_bucket2 +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket1 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2010-10-15).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: hive_test_smb_bucket2 PARTITION(ds=2012-07-02).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ]