diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 371cb0f..2d121a4 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -540,6 +540,7 @@ HIVECONVERTJOINNOCONDITIONALTASK("hive.auto.convert.join.noconditionaltask", true), HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD("hive.auto.convert.join.noconditionaltask.size", 10000000L), + HIVECONVERTJOINUSENONSTAGED("hive.auto.convert.join.use.nonstaged", true), HIVESKEWJOINKEY("hive.skewjoin.key", 100000), HIVESKEWJOINMAPJOINNUMMAPTASK("hive.skewjoin.mapjoin.map.tasks", 10000), HIVESKEWJOINMAPJOINMINSPLIT("hive.skewjoin.mapjoin.min.split", 33554432L), //32M diff --git conf/hive-default.xml.template conf/hive-default.xml.template index 66d22f9..cd2e6d6 100644 --- conf/hive-default.xml.template +++ conf/hive-default.xml.template @@ -866,6 +866,15 @@ + hive.auto.convert.join.use.nonstaged + true + For conditional joins, if input stream from a small alias can be directly applied to join operator without + filtering or projection, the alias need not to be pre-staged in distributed cache via mapred local task. + Currently, this is not working with vectorization or tez execution engine. + + + + hive.script.auto.progress false Whether Hive Transform/Map/Reduce Clause should automatically send progress information to TaskTracker to avoid the task getting killed because of inactivity. Hive sends progress information when the script is outputting to stderr. This option removes the need of periodically producing stderr messages, but users should be cautious because this may prevent infinite loops in the scripts to be killed by TaskTracker. diff --git itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java index 9ad5986..4795cb3 100644 --- itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java +++ itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java @@ -87,7 +87,6 @@ import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer; import org.apache.hadoop.hive.serde2.thrift.test.Complex; import org.apache.hadoop.hive.shims.HadoopShims; -import org.apache.hadoop.hive.shims.Hadoop23Shims; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; @@ -366,14 +365,7 @@ public QTestUtil(String outDir, String logDir, MiniClusterType clusterType, if (clusterType != MiniClusterType.none) { dfs = shims.getMiniDfs(conf, numberOfDataNodes, true, null); FileSystem fs = dfs.getFileSystem(); - if (clusterType == MiniClusterType.tez) { - if (!(shims instanceof Hadoop23Shims)) { - throw new Exception("Cannot run tez on hadoop-1, Version: "+this.hadoopVer); - } - mr = ((Hadoop23Shims)shims).getMiniTezCluster(conf, 4, getHdfsUriString(fs.getUri().toString()), 1); - } else { mr = shims.getMiniMrCluster(conf, 4, getHdfsUriString(fs.getUri().toString()), 1); - } } initConf(); diff --git itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/MapJoinCounterHook.java itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/MapJoinCounterHook.java index 1b0d57e..db1b0d5 100644 --- itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/MapJoinCounterHook.java +++ itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/MapJoinCounterHook.java @@ -39,11 +39,12 @@ public void run(HookContext hookContext) { String queryID = plan.getQueryId(); // String query = SessionState.get().getCmd(); - int convertedMapJoin = 0; int commonJoin = 0; + int hintedMapJoin = 0; + int convertedMapJoin = 0; + int hintedMapJoinLocal = 0; + int convertedMapJoinLocal = 0; int backupCommonJoin = 0; - int convertedLocalMapJoin = 0; - int localMapJoin = 0; List list = hookContext.getCompleteTaskList(); for (TaskRunner tskRunner : list) { @@ -53,23 +54,27 @@ public void run(HookContext hookContext) { case Task.COMMON_JOIN: commonJoin++; break; - case Task.CONVERTED_LOCAL_MAPJOIN: - convertedLocalMapJoin++; + case Task.HINTED_MAPJOIN: + hintedMapJoin++; + break; + case Task.HINTED_MAPJOIN_LOCAL: + hintedMapJoinLocal++; break; case Task.CONVERTED_MAPJOIN: convertedMapJoin++; break; + case Task.CONVERTED_MAPJOIN_LOCAL: + convertedMapJoinLocal++; + break; case Task.BACKUP_COMMON_JOIN: backupCommonJoin++; break; - case Task.LOCAL_MAPJOIN: - localMapJoin++; - break; } } LogHelper console = SessionState.getConsole(); - console.printError("[MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: " + convertedLocalMapJoin - + " CONVERTED_MAPJOIN: " + convertedMapJoin + " LOCAL_MAPJOIN: "+localMapJoin+ " COMMON_JOIN: "+commonJoin + console.printError("[MapJoinCounter PostHook] COMMON_JOIN: " + commonJoin + + " HINTED_MAPJOIN: " + hintedMapJoin + " HINTED_MAPJOIN_LOCAL: " + hintedMapJoinLocal + + " CONVERTED_MAPJOIN: " + convertedMapJoin + " CONVERTED_MAPJOIN_LOCAL: " + convertedMapJoinLocal + " BACKUP_COMMON_JOIN: " + backupCommonJoin); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java index d8f4eb4..3cfaacf 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java @@ -51,7 +51,7 @@ */ protected transient List[] joinKeysStandardObjectInspectors; - protected transient byte posBigTable = -1; // one of the tables that is not in memory + protected transient byte posBigTable = -1; // pos of driver alias protected transient RowContainer> emptyList = null; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableLoader.java ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableLoader.java index a080fcc..988cc57 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableLoader.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableLoader.java @@ -30,7 +30,8 @@ */ public interface HashTableLoader { - void load(ExecMapperContext context, Configuration hconf, MapJoinDesc desc, byte posBigTable, - MapJoinTableContainer[] mapJoinTables, MapJoinTableContainerSerDe[] mapJoinTableSerdes) + void init(ExecMapperContext context, Configuration hconf, MapJoinOperator joinOp); + + void load(MapJoinTableContainer[] mapJoinTables, MapJoinTableContainerSerDe[] mapJoinTableSerdes) throws HiveException; } 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 fc08b28..24f1229 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.exec; import java.io.BufferedOutputStream; +import java.io.IOException; import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.ArrayList; @@ -26,6 +27,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; @@ -153,9 +155,8 @@ protected void initializeOp(Configuration hconf) throws HiveException { inputObjInspectors, posBigTableAlias, tagLen); if (!noOuterJoin) { - List[] rowContainerObjectInspectors = new List[tagLen]; for (Byte alias : order) { - if (alias == posBigTableAlias) { + if (alias == posBigTableAlias || joinValues[alias] == null) { continue; } List rcOIs = joinValuesObjectInspectors[alias]; @@ -164,7 +165,6 @@ protected void initializeOp(Configuration hconf) throws HiveException { rcOIs = new ArrayList(rcOIs); rcOIs.add(PrimitiveObjectInspectorFactory.writableShortObjectInspector); } - rowContainerObjectInspectors[alias] = rcOIs; } } mapJoinTables = new MapJoinTableContainer[tagLen]; @@ -198,7 +198,9 @@ protected void initializeOp(Configuration hconf) throws HiveException { } } - + public MapJoinTableContainer[] getMapJoinTables() { + return mapJoinTables; + } private static List[] getStandardObjectInspectors( List[] aliasToObjectInspectors, int maxTag) { @@ -265,33 +267,7 @@ private boolean hasFilter(int alias) { public void closeOp(boolean abort) throws HiveException { try { if (mapJoinTables != null) { - // get tmp path - Path tmpPath = this.getExecContext().getLocalWork().getTmpPath(); - LOG.info("Temp URI for side table: " + tmpPath); - for (byte tag = 0; tag < mapJoinTables.length; tag++) { - // get the key and value - MapJoinTableContainer tableContainer = mapJoinTables[tag]; - if (tableContainer == null) { - continue; - } - // get current input file name - 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 dumpFilePrefix = conf.getDumpFilePrefix(); - Path path = Utilities.generatePath(tmpPath, dumpFilePrefix, tag, fileName); - console.printInfo(Utilities.now() + "\tDump the side-table into file: " + path); - // get the hashtable file and path - FileSystem fs = path.getFileSystem(hconf); - ObjectOutputStream out = new ObjectOutputStream(new BufferedOutputStream(fs.create(path), 4096)); - try { - mapJoinTableSerdes[tag].persist(out, tableContainer); - } finally { - out.close(); - } - tableContainer.clear(); - console.printInfo(Utilities.now() + "\tUpload 1 File to: " + path); - } + flushToFile(); } super.closeOp(abort); } catch (Exception e) { @@ -299,6 +275,38 @@ public void closeOp(boolean abort) throws HiveException { } } + protected void flushToFile() throws IOException, HiveException { + // get tmp file URI + Path tmpURI = getExecContext().getLocalWork().getTmpPath(); + LOG.info("Temp URI for side table: " + tmpURI); + for (byte tag = 0; tag < mapJoinTables.length; tag++) { + // get the key and value + MapJoinTableContainer tableContainer = mapJoinTables[tag]; + if (tableContainer == null) { + continue; + } + // get current input file name + 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 dumpFilePrefix = conf.getDumpFilePrefix(); + Path path = Utilities.generatePath(tmpURI, dumpFilePrefix, tag, fileName); + console.printInfo(Utilities.now() + "\tDump the side-table into file: " + path); + // get the hashtable file and path + FileSystem fs = path.getFileSystem(hconf); + ObjectOutputStream out = new ObjectOutputStream(new BufferedOutputStream(fs.create(path), 4096)); + try { + mapJoinTableSerdes[tag].persist(out, tableContainer); + } finally { + out.close(); + } + tableContainer.clear(); + FileStatus status = fs.getFileStatus(path); + console.printInfo(Utilities.now() + "\tUploaded 1 File to: " + path + + " (" + status.getLen() + " bytes)"); + } + } + /** * Implements the getName function for the Node Interface. * diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java index 1e0314d..46e37c2 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java @@ -53,11 +53,13 @@ ObjectInspector[] inputObjInspector, int posBigTableAlias, int tagLen) throws HiveException { List[] result = new List[tagLen]; - for (byte alias = 0; alias < exprEntries.length; alias++) { - //get big table - if (alias == (byte) posBigTableAlias){ - //skip the big tables - continue; + + int iterate = Math.min(exprEntries.length, inputObjInspector.length); + for (byte alias = 0; alias < iterate; alias++) { + if (alias == (byte) posBigTableAlias || + exprEntries[alias] == null || inputObjInspector[alias] == null) { + // skip the driver and directly loadable tables + continue; } List exprList = exprEntries[alias]; @@ -77,7 +79,7 @@ List[] result = new List[tagLen]; for (byte alias = 0; alias < aliasToObjectInspectors.length; alias++) { //get big table - if(alias == (byte) posBigTableAlias ){ + if(alias == (byte) posBigTableAlias || aliasToObjectInspectors[alias] == null){ //skip the big tables continue; } @@ -106,6 +108,9 @@ public static int populateJoinKeyValue(List[] outMap, int posBigTableAlias) throws HiveException { int total = 0; for (Entry> e : inputMap.entrySet()) { + if (e.getValue() == null) { + continue; + } Byte key = order == null ? e.getKey() : order[e.getKey()]; List valueFields = new ArrayList(); for (ExprNodeDesc expr : e.getValue()) { 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 bdc85b9..c0f4cd7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java @@ -145,8 +145,8 @@ private void loadHashTable() throws HiveException { } } perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.LOAD_HASHTABLE); - loader.load(this.getExecContext(), hconf, this.getConf(), - posBigTable, mapJoinTables, mapJoinTableSerdes); + loader.init(getExecContext(), hconf, this); + loader.load(mapJoinTables, mapJoinTableSerdes); cache.cache(tableKey, mapJoinTables); cache.cache(serdeKey, mapJoinTableSerdes); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.LOAD_HASHTABLE); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java index 56676df..c5bbe68 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java @@ -70,15 +70,16 @@ public static final int NO_TAG = 0; public static final int COMMON_JOIN = 1; - public static final int CONVERTED_MAPJOIN = 2; - public static final int CONVERTED_LOCAL_MAPJOIN = 3; - public static final int BACKUP_COMMON_JOIN = 4; - public static final int LOCAL_MAPJOIN = 5; + public static final int HINTED_MAPJOIN = 2; + public static final int HINTED_MAPJOIN_LOCAL = 3; + public static final int CONVERTED_MAPJOIN = 4; + public static final int CONVERTED_MAPJOIN_LOCAL = 5; + public static final int BACKUP_COMMON_JOIN = 6; // The join task is converted to a mapjoin task. This can only happen if // hive.auto.convert.join.noconditionaltask is set to true. No conditional task was // created in case the mapjoin failed. - public static final int MAPJOIN_ONLY_NOBACKUP = 6; - public static final int CONVERTED_SORTMERGEJOIN = 7; + public static final int MAPJOIN_ONLY_NOBACKUP = 7; + public static final int CONVERTED_SORTMERGEJOIN = 8; // Descendants tasks who subscribe feeds from this task protected transient List> feedSubscribers; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/TemporaryHashSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/TemporaryHashSinkOperator.java new file mode 100644 index 0000000..16baf4b --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/TemporaryHashSinkOperator.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.HashTableSinkDesc; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; + +import java.io.IOException; + +public class TemporaryHashSinkOperator extends HashTableSinkOperator { + public TemporaryHashSinkOperator(MapJoinDesc desc) { + conf = new HashTableSinkDesc(desc); + } + + @Override + protected void flushToFile() throws IOException, HiveException { + // do nothing + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java index 22e5777..54e1dd8 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java @@ -327,7 +327,7 @@ public int execute(DriverContext driverContext) { try{ MapredLocalWork localwork = mWork.getMapLocalWork(); - if (localwork != null) { + if (localwork != null && localwork.hasStagedAlias()) { if (!ShimLoader.getHadoopShims().isLocalMode(job)) { Path localPath = localwork.getTmpPath(); Path hdfsPath = mWork.getTmpHDFSPath(); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java index 58484af..5cf347b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java @@ -20,6 +20,9 @@ import java.io.BufferedInputStream; import java.io.FileInputStream; import java.io.ObjectInputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -27,13 +30,20 @@ import org.apache.hadoop.filecache.DistributedCache; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.HashTableSinkOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.TemporaryHashSinkOperator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer; import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainerSerDe; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; +import org.apache.hadoop.hive.ql.plan.MapredLocalWork; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.mapred.JobConf; /** * HashTableLoader for MR loads the hashtable for MapJoins from local disk (hashtables @@ -44,46 +54,41 @@ private static final Log LOG = LogFactory.getLog(MapJoinOperator.class.getName()); - public HashTableLoader() { + private ExecMapperContext context; + private Configuration hconf; + + private MapJoinOperator joinOp; + private MapJoinDesc desc; + + @Override + public void init(ExecMapperContext context, Configuration hconf, MapJoinOperator joinOp) { + this.context = context; + this.hconf = hconf; + this.joinOp = joinOp; + this.desc = joinOp.getConf(); } @Override - public void load(ExecMapperContext context, - Configuration hconf, - MapJoinDesc desc, - byte posBigTable, + public void load( MapJoinTableContainer[] mapJoinTables, MapJoinTableContainerSerDe[] mapJoinTableSerdes) throws HiveException { - Path baseDir = null; - Path currentInputPath = context.getCurrentInputPath(); - LOG.info("******* Load from HashTable File: input : " + currentInputPath); - String fileName = context.getLocalWork().getBucketFileName(currentInputPath.toString()); + String currentInputPath = context.getCurrentInputPath().toString(); + LOG.info("******* Load from HashTable for input file: " + currentInputPath); + MapredLocalWork localWork = context.getLocalWork(); try { - if (ShimLoader.getHadoopShims().isLocalMode(hconf)) { - baseDir = context.getLocalWork().getTmpPath(); - } else { - Path[] localArchives; - String stageID = context.getLocalWork().getStageID(); - String suffix = Utilities.generateTarFileName(stageID); - FileSystem localFs = FileSystem.getLocal(hconf); - localArchives = DistributedCache.getLocalCacheArchives(hconf); - Path archive; - for (int j = 0; j < localArchives.length; j++) { - archive = localArchives[j]; - if (!archive.getName().endsWith(suffix)) { - continue; - } - baseDir = archive.makeQualified(localFs); - } + if (localWork.getDirectFetchOp() != null) { + loadDirectly(mapJoinTables, currentInputPath); + } + Path baseDir = getBaseDir(localWork); + if (baseDir == null) { + return; } + String fileName = localWork.getBucketFileName(currentInputPath); for (int pos = 0; pos < mapJoinTables.length; pos++) { - if (pos == posBigTable) { + if (pos == desc.getPosBigTable() || mapJoinTables[pos] != null) { continue; } - if(baseDir == null) { - throw new IllegalStateException("baseDir cannot be null"); - } Path path = Utilities.generatePath(baseDir, desc.getDumpFilePrefix(), (byte)pos, fileName); LOG.info("\tLoad back 1 hashtable file from tmp file uri:" + path); ObjectInputStream in = new ObjectInputStream(new BufferedInputStream( @@ -99,4 +104,54 @@ public void load(ExecMapperContext context, } } + private Path getBaseDir(MapredLocalWork localWork) throws Exception { + if (ShimLoader.getHadoopShims().isLocalMode(hconf)) { + return localWork.getTmpPath(); + } + Path[] localArchives = DistributedCache.getLocalCacheArchives(hconf); + if (localArchives != null) { + String stageID = localWork.getStageID(); + String suffix = Utilities.generateTarFileName(stageID); + FileSystem localFs = FileSystem.getLocal(hconf); + for (int j = 0; j < localArchives.length; j++) { + Path archive = localArchives[j]; + if (!archive.getName().endsWith(suffix)) { + continue; + } + return archive.makeQualified(localFs); + } + } + return null; + } + + private void loadDirectly(MapJoinTableContainer[] mapJoinTables, String inputFileName) + throws Exception { + MapredLocalWork localWork = context.getLocalWork(); + List> directWorks = localWork.getDirectFetchOp().get(joinOp); + if (directWorks == null || directWorks.isEmpty()) { + return; + } + JobConf job = new JobConf(hconf); + MapredLocalTask localTask = new MapredLocalTask(localWork, job, false); + + HashTableSinkOperator sink = new TemporaryHashSinkOperator(desc); + sink.setParentOperators(new ArrayList>(directWorks)); + + for (Operator operator : directWorks) { + if (operator instanceof TableScanOperator) { + operator.setChildOperators(Arrays.>asList(sink)); + } + } + localTask.setExecContext(context); + localTask.startForward(inputFileName); + + MapJoinTableContainer[] tables = sink.getMapJoinTables(); + for (int i = 0; i < sink.getNumParent(); i++) { + if (sink.getParentOperators().get(i) != null) { + mapJoinTables[i] = tables[i]; + } + } + + Arrays.fill(tables, null); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java index 2d2508d..29fbb03 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java @@ -81,7 +81,7 @@ */ public class MapredLocalTask extends Task implements Serializable { - private Map fetchOperators; + private Map fetchOperators = new HashMap(); protected HadoopJobExecHelper jobExecHelper; private JobConf job; public static transient final Log l4j = LogFactory.getLog(MapredLocalTask.class); @@ -93,7 +93,7 @@ // not sure we need this exec context; but all the operators in the work // will pass this context throught - private final ExecMapperContext execContext = new ExecMapperContext(); + private ExecMapperContext execContext = new ExecMapperContext(); private Process executor; @@ -107,6 +107,10 @@ public MapredLocalTask(MapredLocalWork plan, JobConf job, boolean isSilent) thro console = new LogHelper(LOG, isSilent); } + public void setExecContext(ExecMapperContext execContext) { + this.execContext = execContext; + } + @Override public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) { super.initialize(conf, queryPlan, driverContext); @@ -291,26 +295,11 @@ public int executeFromChildJVM(DriverContext driverContext) { console.printInfo(Utilities.now() + "\tStarting to launch local task to process map join;\tmaximum memory = " + memoryMXBean.getHeapMemoryUsage().getMax()); - fetchOperators = new HashMap(); - Map fetchOpJobConfMap = new HashMap(); execContext.setJc(job); // set the local work, so all the operator can get this context execContext.setLocalWork(work); - boolean inputFileChangeSenstive = work.getInputFileChangeSensitive(); try { - - initializeOperators(fetchOpJobConfMap); - // for each big table's bucket, call the start forward - if (inputFileChangeSenstive) { - for (Map> bigTableBucketFiles : work - .getBucketMapjoinContext().getAliasBucketFileNameMapping().values()) { - for (String bigTableBucket : bigTableBucketFiles.keySet()) { - startForward(inputFileChangeSenstive, bigTableBucket); - } - } - } else { - startForward(inputFileChangeSenstive, null); - } + startForward(null); long currentTime = System.currentTimeMillis(); long elapsed = currentTime - startTime; console.printInfo(Utilities.now() + "\tEnd of local task; Time Taken: " @@ -328,6 +317,26 @@ public int executeFromChildJVM(DriverContext driverContext) { return 0; } + public void startForward(String bigTableBucket) throws Exception { + boolean inputFileChangeSenstive = work.getInputFileChangeSensitive(); + initializeOperators(new HashMap()); + // for each big table's bucket, call the start forward + if (inputFileChangeSenstive) { + for (Map> bigTableBucketFiles : work + .getBucketMapjoinContext().getAliasBucketFileNameMapping().values()) { + if (bigTableBucket == null) { + for (String bigTableBucketFile : bigTableBucketFiles.keySet()) { + startForward(inputFileChangeSenstive, bigTableBucketFile); + } + } else if (bigTableBucketFiles.keySet().contains(bigTableBucket)) { + startForward(inputFileChangeSenstive, bigTableBucket); + } + } + } else { + startForward(inputFileChangeSenstive, null); + } + } + private void startForward(boolean inputFileChangeSenstive, String bigTableBucket) throws Exception { for (Map.Entry entry : fetchOperators.entrySet()) { @@ -348,24 +357,18 @@ private void startForward(boolean inputFileChangeSenstive, String bigTableBucket // get the root operator Operator forwardOp = work.getAliasToWork().get(alias); // walk through the operator tree - while (true) { + while (!forwardOp.getDone()) { InspectableObject row = fetchOp.getNextRow(); if (row == null) { - if (inputFileChangeSenstive) { - execContext.setCurrentBigBucketFile(bigTableBucket); - forwardOp.reset(); - } - forwardOp.close(false); break; } forwardOp.processOp(row.o, 0); - // check if any operator had a fatal error or early exit during - // execution - if (forwardOp.getDone()) { - // ExecMapper.setDone(true); - break; - } } + if (inputFileChangeSenstive) { + execContext.setCurrentBigBucketFile(bigTableBucket); + forwardOp.reset(); + } + forwardOp.close(false); } } @@ -373,6 +376,9 @@ private void initializeOperators(Map fetchOpJobConfMap) throws HiveException { // this mapper operator is used to initialize all the operators for (Map.Entry entry : work.getAliasToFetchWork().entrySet()) { + if (entry.getValue() == null) { + continue; + } JobConf jobClone = new JobConf(job); TableScanOperator ts = (TableScanOperator)work.getAliasToWork().get(entry.getKey()); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java index 2df8ab9..47f9d21 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java @@ -47,14 +47,19 @@ private static final Log LOG = LogFactory.getLog(MapJoinOperator.class.getName()); - public HashTableLoader() { + private ExecMapperContext context; + private Configuration hconf; + private MapJoinDesc desc; + + @Override + public void init(ExecMapperContext context, Configuration hconf, MapJoinOperator joinOp) { + this.context = context; + this.hconf = hconf; + this.desc = joinOp.getConf(); } @Override - public void load(ExecMapperContext context, - Configuration hconf, - MapJoinDesc desc, - byte posBigTable, + public void load( MapJoinTableContainer[] mapJoinTables, MapJoinTableContainerSerDe[] mapJoinTableSerdes) throws HiveException { @@ -65,7 +70,7 @@ public void load(ExecMapperContext context, HiveConf.ConfVars.HIVEHASHTABLELOADFACTOR); for (int pos = 0; pos < mapJoinTables.length; pos++) { - if (pos == posBigTable) { + if (pos == desc.getPosBigTable()) { continue; } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java index 5a53e15..726d315 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java @@ -118,6 +118,8 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, Object.. // mapjoin should not affected by join reordering mapJoinOp.getConf().resetOrder(); + HiveConf conf = context.getParseCtx().getConf(); + HashTableSinkDesc hashTableSinkDesc = new HashTableSinkDesc(mapJoinOp.getConf()); HashTableSinkOperator hashTableSinkOp = (HashTableSinkOperator) OperatorFactory .get(hashTableSinkDesc); @@ -125,34 +127,59 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, Object.. // set hashtable memory usage float hashtableMemoryUsage; if (context.isFollowedByGroupBy()) { - hashtableMemoryUsage = context.getParseCtx().getConf().getFloatVar( + hashtableMemoryUsage = conf.getFloatVar( HiveConf.ConfVars.HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE); } else { - hashtableMemoryUsage = context.getParseCtx().getConf().getFloatVar( + hashtableMemoryUsage = conf.getFloatVar( HiveConf.ConfVars.HIVEHASHTABLEMAXMEMORYUSAGE); } hashTableSinkOp.getConf().setHashtableMemoryUsage(hashtableMemoryUsage); // get the last operator for processing big tables int bigTable = mapJoinOp.getConf().getPosBigTable(); + Byte[] orders = mapJoinOp.getConf().getTagOrder(); + + // todo: support tez/vectorization + boolean useNontaged = conf.getBoolVar( + HiveConf.ConfVars.HIVECONVERTJOINUSENONSTAGED) && + conf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("mr") && + !conf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED); // the parent ops for hashTableSinkOp List> smallTablesParentOp = new ArrayList>(); List> dummyOperators = new ArrayList>(); + List> directOperators = + new ArrayList>(); // get all parents List> parentsOp = mapJoinOp.getParentOperators(); for (int i = 0; i < parentsOp.size(); i++) { if (i == bigTable) { smallTablesParentOp.add(null); + directOperators.add(null); continue; } Operator parent = parentsOp.get(i); + boolean directFetchable = useNontaged && + (parent instanceof TableScanOperator || parent instanceof MapJoinOperator); + if (directFetchable) { + // no filter, no projection. no need to stage + smallTablesParentOp.add(null); + directOperators.add(parent); + hashTableSinkDesc.getKeys().put(orders[i], null); + hashTableSinkDesc.getExprs().put(orders[i], null); + hashTableSinkDesc.getFilters().put(orders[i], null); + } else { + // keep the parent id correct + smallTablesParentOp.add(parent); + directOperators.add(null); + } // let hashtable Op be the child of this parent parent.replaceChild(mapJoinOp, hashTableSinkOp); - // keep the parent id correct - smallTablesParentOp.add(parent); + if (directFetchable) { + parent.setChildOperators(null); + } // create an new operator: HashTable DummyOpeator, which share the table desc HashTableDummyDesc desc = new HashTableDummyDesc(); @@ -186,6 +213,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, Object.. for (Operator op : dummyOperators) { context.addDummyParentOp(op); } + context.addDirectWorks(mapJoinOp, directOperators); return null; } 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 83b8d6e..82a833e 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 @@ -118,23 +118,31 @@ private void processCurrentTask(Task currTask, currTask.setBackupTask(null); if (currTask.getTaskTag() == Task.CONVERTED_MAPJOIN) { - localTask.setTaskTag(Task.CONVERTED_LOCAL_MAPJOIN); + localTask.setTaskTag(Task.CONVERTED_MAPJOIN_LOCAL); } else { - localTask.setTaskTag(Task.LOCAL_MAPJOIN); + localTask.setTaskTag(Task.HINTED_MAPJOIN_LOCAL); + currTask.setTaskTag(Task.HINTED_MAPJOIN); } // replace the map join operator to local_map_join operator in the operator tree // and return all the dummy parent - LocalMapJoinProcCtx localMapJoinProcCtx= adjustLocalTask(localTask); + LocalMapJoinProcCtx localMapJoinProcCtx = adjustLocalTask(localTask); List> dummyOps = localMapJoinProcCtx.getDummyParentOp(); // create new local work and setup the dummy ops - MapredLocalWork newLocalWork = new MapredLocalWork(); + MapredLocalWork newLocalWork = localwork.extractDirectWorks( + localMapJoinProcCtx.getDirectWorks()); newLocalWork.setDummyParentOp(dummyOps); - newLocalWork.setTmpPath(tmpPath); - newLocalWork.setInputFileChangeSensitive(localwork.getInputFileChangeSensitive()); - newLocalWork.setBucketMapjoinContext(localwork.copyPartSpecMappingOnly()); mapredWork.getMapWork().setMapLocalWork(newLocalWork); + + if (localwork.getAliasToFetchWork().isEmpty()) { + // no alias to stage.. no local task + newLocalWork.setHasStagedAlias(false); + currTask.setBackupTask(localTask.getBackupTask()); + currTask.setBackupChildrenTasks(localTask.getBackupChildrenTasks()); + return; + } + newLocalWork.setHasStagedAlias(true); // get all parent tasks List> parentTasks = currTask.getParentTasks(); currTask.setParentTasks(null); @@ -160,7 +168,7 @@ private void processCurrentTask(Task currTask, // set list work List listWork = (List) conditionalWork.getListWorks(); index = listWork.indexOf(mapredWork); - listWork.set(index, (Serializable) localwork); + listWork.set(index, localwork); conditionalWork.setListWorks(listWork); ConditionalResolver resolver = conditionalTask.getResolver(); if (resolver instanceof ConditionalResolverSkewJoin) { @@ -272,10 +280,13 @@ public void setPhysicalContext(PhysicalContext physicalContext) { private List> dummyParentOp = null; private boolean isFollowedByGroupBy; + private Map>> directWorks; + public LocalMapJoinProcCtx(Task task, ParseContext parseCtx) { currentTask = task; this.parseCtx = parseCtx; dummyParentOp = new ArrayList>(); + directWorks = new HashMap>>(); isFollowedByGroupBy = false; } @@ -313,5 +324,19 @@ public void setDummyParentOp(List> op) { public void addDummyParentOp(Operator op) { this.dummyParentOp.add(op); } + + public void setDirectWorks( + Map>> directWorks) { + this.directWorks = directWorks; + } + + public Map>> getDirectWorks() { + return directWorks; + } + + public void addDirectWorks( + MapJoinOperator mapJoinOp, List> directOperators) { + directWorks.put(mapJoinOp, directOperators); + } } } diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverCommonJoin.java ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverCommonJoin.java index ebccb14..5c3a582 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverCommonJoin.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverCommonJoin.java @@ -142,7 +142,7 @@ public ConditionalResolverCommonJoin() { // run the map join task Task task = ctx.getAliasToTask().get(bigTableAlias); //set task tag - if(task.getTaskTag() == Task.CONVERTED_LOCAL_MAPJOIN) { + if (task.getBackupTask() != null) { task.getBackupTask().setTaskTag(Task.BACKUP_COMMON_JOIN); } resTsks.add(task); 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 c30da56..f00192d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java @@ -92,7 +92,7 @@ public HashTableSinkDesc() { public HashTableSinkDesc(MapJoinDesc clone) { this.bigKeysDirMap = clone.getBigKeysDirMap(); this.conds = clone.getConds(); - this.exprs= clone.getExprs(); + this.exprs = new HashMap>(clone.getExprs()); this.handleSkewJoin = clone.getHandleSkewJoin(); this.keyTableDesc = clone.getKeyTableDesc(); this.noOuterJoin = clone.getNoOuterJoin(); @@ -102,10 +102,10 @@ public HashTableSinkDesc(MapJoinDesc clone) { this.skewKeysValuesTables = clone.getSkewKeysValuesTables(); this.smallKeysDirMap = clone.getSmallKeysDirMap(); this.tagOrder = clone.getTagOrder(); - this.filters = clone.getFilters(); + this.filters = new HashMap>(clone.getFilters()); this.filterMap = clone.getFilterMap(); - this.keys = clone.getKeys(); + this.keys = new HashMap>(clone.getKeys()); this.keyTblDesc = clone.getKeyTblDesc(); this.valueTblDescs = clone.getValueTblDescs(); this.valueTblFilteredDescs = clone.getValueFilteredTblDescs(); 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 709c50e..d566087 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java @@ -19,10 +19,16 @@ package org.apache.hadoop.hive.ql.plan; import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; /** @@ -40,7 +46,10 @@ private Path tmpPath; private String stageID; - private List> dummyParentOp ; + private List> dummyParentOp; + private Map>> directFetchOp; + + private boolean hasStagedAlias; public MapredLocalWork() { @@ -177,4 +186,61 @@ private String getFileName(String path) { } return path.substring(last_separator + 1); } + + public MapredLocalWork extractDirectWorks( + Map>> directWorks) { + MapredLocalWork newLocalWork = new MapredLocalWork(); + newLocalWork.setTmpPath(tmpPath); + newLocalWork.setInputFileChangeSensitive(inputFileChangeSensitive); + + Set> validWorks = getDirectWorks(directWorks.values()); + if (validWorks.isEmpty()) { + // all small aliases are staged.. no need full bucket context + newLocalWork.setBucketMapjoinContext(copyPartSpecMappingOnly()); + return newLocalWork; + } + newLocalWork.directFetchOp = + new HashMap>>(directWorks); + newLocalWork.aliasToWork = new LinkedHashMap>(); + newLocalWork.aliasToFetchWork = new LinkedHashMap(); + + Map> works = new HashMap>(aliasToWork); + for (Map.Entry> entry : works.entrySet()) { + String alias = entry.getKey(); + boolean notStaged = validWorks.contains(entry.getValue()); + newLocalWork.aliasToWork.put(alias, notStaged ? aliasToWork.remove(alias) : null); + newLocalWork.aliasToFetchWork.put(alias, notStaged ? aliasToFetchWork.remove(alias) : null); + } + // copy full bucket context + newLocalWork.setBucketMapjoinContext(getBucketMapjoinContext()); + return newLocalWork; + } + + private Set> getDirectWorks(Collection>> values) { + Set> operators = new HashSet>(); + for (List> works : values) { + for (Operator work : works) { + if (work != null) { + operators.add(work); + } + } + } + return operators; + } + + public void setDirectFetchOp(Map>> op){ + this.directFetchOp = op; + } + + public Map>> getDirectFetchOp() { + return directFetchOp; + } + + public boolean hasStagedAlias() { + return hasStagedAlias; + } + + public void setHasStagedAlias(boolean hasStagedAlias) { + this.hasStagedAlias = hasStagedAlias; + } } diff --git ql/src/test/queries/clientpositive/auto_join_without_localtask.q ql/src/test/queries/clientpositive/auto_join_without_localtask.q new file mode 100644 index 0000000..bb7edc9 --- /dev/null +++ ql/src/test/queries/clientpositive/auto_join_without_localtask.q @@ -0,0 +1,28 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.use.nonstaged=true; + +set hive.auto.convert.join.noconditionaltask.size=100; + +explain +select a.* from src a join src b on a.key=b.key limit 40; + +select a.* from src a join src b on a.key=b.key limit 40; + +explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40; + +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40; + +set hive.auto.convert.join.noconditionaltask.size=100; + +explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + +set hive.mapjoin.localtask.max.memory.usage = 0.0001; +set hive.mapjoin.check.memory.rows = 2; + +-- fallback to common join +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + diff --git ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out index 0595cd6..25d7b57 100644 --- ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out +++ ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out @@ -165,31 +165,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) ds) "2008-04-08")) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -226,6 +205,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out index fa261b3..d888153 100644 --- ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out +++ ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out @@ -1,14 +1,2 @@ -#### A masked pattern was here #### -PREHOOK: type: CREATETABLE -#### A masked pattern was here #### -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: default@header_footer_table_1 -PREHOOK: query: SELECT * FROM header_footer_table_1 -PREHOOK: type: QUERY -PREHOOK: Input: default@header_footer_table_1 -#### A masked pattern was here #### -POSTHOOK: query: SELECT * FROM header_footer_table_1 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@header_footer_table_1 -#### A masked pattern was here #### -Failed with exception java.io.IOException:java.io.IOException: footer number exceeds the limit defined in hive.file.max.footer +Command failed with exit code = -1 +Query returned non-zero code: -1, cause: null diff --git ql/src/test/results/clientnegative/sortmerge_mapjoin_mismatch_1.q.out ql/src/test/results/clientnegative/sortmerge_mapjoin_mismatch_1.q.out index bca069a..0f8deba 100644 --- ql/src/test/results/clientnegative/sortmerge_mapjoin_mismatch_1.q.out +++ ql/src/test/results/clientnegative/sortmerge_mapjoin_mismatch_1.q.out @@ -62,31 +62,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME table_asc) a) (TOK_TABREF (TOK_TABNAME table_desc) 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 a))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -125,6 +104,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/auto_join1.q.out ql/src/test/results/clientpositive/auto_join1.q.out index b93c10f..2ea1806 100644 --- ql/src/test/results/clientpositive/auto_join1.q.out +++ ql/src/test/results/clientpositive/auto_join1.q.out @@ -15,32 +15,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) value))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-4 depends on stages: Stage-5 + Stage-4 is a root stage Stage-0 depends on stages: Stage-4 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -76,6 +55,14 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 Stage: Stage-0 Move Operator diff --git ql/src/test/results/clientpositive/auto_join15.q.out ql/src/test/results/clientpositive/auto_join15.q.out index 6bfcfc7..01a67f5 100644 --- ql/src/test/results/clientpositive/auto_join15.q.out +++ ql/src/test/results/clientpositive/auto_join15.q.out @@ -18,32 +18,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key) k1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) value) v1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) key) k2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) value) v2)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL k1)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL v1)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL k2)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL v2))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL a) k1) (. (TOK_TABLE_OR_COL a) v1) (. (TOK_TABLE_OR_COL a) k2) (. (TOK_TABLE_OR_COL a) v2))))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-2 depends on stages: Stage-6 + Stage-2 is a root stage Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - a:src1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a:src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -96,6 +75,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a:src1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:src1 + TableScan + alias: src1 Reduce Operator Tree: Extract Select Operator diff --git ql/src/test/results/clientpositive/auto_join17.q.out ql/src/test/results/clientpositive/auto_join17.q.out index 698270e..fdff096 100644 --- ql/src/test/results/clientpositive/auto_join17.q.out +++ ql/src/test/results/clientpositive/auto_join17.q.out @@ -15,32 +15,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest1))) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME src1))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME src2)))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-4 depends on stages: Stage-5 + Stage-4 is a root stage Stage-0 depends on stages: Stage-4 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -80,6 +59,14 @@ STAGE PLANS: name: default.dest1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 Stage: Stage-0 Move Operator diff --git ql/src/test/results/clientpositive/auto_join19.q.out ql/src/test/results/clientpositive/auto_join19.q.out index 247b081..1d9820a 100644 --- ql/src/test/results/clientpositive/auto_join19.q.out +++ ql/src/test/results/clientpositive/auto_join19.q.out @@ -17,32 +17,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcpart) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) value))) (TOK_WHERE (and (or (= (. (TOK_TABLE_OR_COL src1) ds) '2008-04-08') (= (. (TOK_TABLE_OR_COL src1) ds) '2008-04-09')) (or (= (. (TOK_TABLE_OR_COL src1) hr) '12') (= (. (TOK_TABLE_OR_COL src1) hr) '11')))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-4 depends on stages: Stage-5 + Stage-4 is a root stage Stage-0 depends on stages: Stage-4 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - src2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src2 - TableScan - alias: src2 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -78,6 +57,14 @@ STAGE PLANS: name: default.dest1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src2 + TableScan + alias: src2 Stage: Stage-0 Move Operator diff --git ql/src/test/results/clientpositive/auto_join2.q.out ql/src/test/results/clientpositive/auto_join2.q.out index 756cfa1..592566b 100644 --- ql/src/test/results/clientpositive/auto_join2.q.out +++ ql/src/test/results/clientpositive/auto_join2.q.out @@ -15,47 +15,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key))) (TOK_TABREF (TOK_TABNAME src) src3) (= (+ (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)) (. (TOK_TABLE_OR_COL src3) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j2))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src3) value))))) STAGE DEPENDENCIES: - Stage-8 is a root stage - Stage-6 depends on stages: Stage-8 + Stage-6 is a root stage Stage-0 depends on stages: Stage-6 Stage-3 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 - src3 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - src3 - TableScan - alias: src3 - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 {value} - handleSkewJoin: false - keys: - 0 [GenericUDFOPPlus(Column[_col0], Column[_col4])] - 1 [GenericUDFBridge(Column[key])] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -103,6 +67,20 @@ STAGE PLANS: name: default.dest_j2 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + src3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 + src3 + TableScan + alias: src3 Stage: Stage-0 Move Operator diff --git ql/src/test/results/clientpositive/auto_join21.q.out ql/src/test/results/clientpositive/auto_join21.q.out index 053475f..87e491c 100644 --- ql/src/test/results/clientpositive/auto_join21.q.out +++ ql/src/test/results/clientpositive/auto_join21.q.out @@ -16,31 +16,10 @@ STAGE PLANS: Stage: Stage-6 Map Reduce Local Work Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 src2 Fetch Operator limit: -1 Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - filter predicates: - 0 {(key < 10)} - 1 - 2 {(key < 10)} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 src2 TableScan alias: src2 @@ -50,16 +29,16 @@ STAGE PLANS: type: boolean HashTable Sink Operator condition expressions: - 0 {key} {value} + 0 1 {key} {value} 2 {key} {value} filter predicates: - 0 {(key < 10)} + 0 1 2 {(key < 10)} handleSkewJoin: false keys: - 0 [Column[key]] + 0 1 [Column[key]] 2 [Column[key]] Position of Big Table: 2 @@ -135,6 +114,16 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + src2 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 + src2 Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/auto_join22.q.out ql/src/test/results/clientpositive/auto_join22.q.out index 6e5ae4c..7771456 100644 --- ql/src/test/results/clientpositive/auto_join22.q.out +++ ql/src/test/results/clientpositive/auto_join22.q.out @@ -8,46 +8,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) src4) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME src2))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key) src1_key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) value) src1_value)))) src3) (= (. (TOK_TABLE_OR_COL src3) src1_key) (. (TOK_TABLE_OR_COL src4) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME src3))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src4) value) src4_value) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src4) key) src4_key)))) src5)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL src5) src1_value))))))) STAGE DEPENDENCIES: - Stage-8 is a root stage - Stage-3 depends on stages: Stage-8 + Stage-3 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - src5:src3:src1 - Fetch Operator - limit: -1 - src5:src4 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src5:src3:src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - src5:src4 - TableScan - alias: src4 - HashTable Sink Operator - condition expressions: - 0 - 1 {_col3} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[_col2]] - Position of Big Table: 1 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -104,6 +68,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src5:src3:src1 + Fetch Operator + limit: -1 + src5:src4 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src5:src3:src1 + TableScan + alias: src1 + src5:src4 + TableScan + alias: src4 Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/auto_join24.q.out ql/src/test/results/clientpositive/auto_join24.q.out index dbecb2c..b942d68 100644 --- ql/src/test/results/clientpositive/auto_join24.q.out +++ ql/src/test/results/clientpositive/auto_join24.q.out @@ -27,31 +27,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME tst1) a) (TOK_TABREF (TOK_TABNAME tst1) 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_SELEXPR (TOK_FUNCTION sum (. (TOK_TABLE_OR_COL a) cnt)))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-2 depends on stages: Stage-5 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {cnt} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -89,6 +68,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/auto_join25.q.out ql/src/test/results/clientpositive/auto_join25.q.out index d6d0c90..9fe6049 100644 --- ql/src/test/results/clientpositive/auto_join25.q.out +++ ql/src/test/results/clientpositive/auto_join25.q.out @@ -16,17 +16,17 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@dest1 -Execution failed with exit status: 3 +Execution failed with exit status: 2 Obtaining error information Task failed! Task ID: - Stage-7 + Stage-4 Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value @@ -64,29 +64,29 @@ INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j2 -Execution failed with exit status: 3 +Execution failed with exit status: 2 Obtaining error information Task failed! Task ID: - Stage-15 + Stage-10 Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask -Execution failed with exit status: 3 +Execution failed with exit status: 2 Obtaining error information Task failed! Task ID: - Stage-12 + Stage-6 Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value @@ -124,17 +124,17 @@ INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j1 -Execution failed with exit status: 3 +Execution failed with exit status: 2 Obtaining error information Task failed! Task ID: - Stage-8 + Stage-5 Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value diff --git ql/src/test/results/clientpositive/auto_join26.q.out ql/src/test/results/clientpositive/auto_join26.q.out index 7ca6bb9..d7c194e 100644 --- ql/src/test/results/clientpositive/auto_join26.q.out +++ ql/src/test/results/clientpositive/auto_join26.q.out @@ -15,32 +15,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-2 depends on stages: Stage-6 + Stage-2 is a root stage Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -87,6 +66,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/auto_join28.q.out ql/src/test/results/clientpositive/auto_join28.q.out index 488e15c..0a65f34 100644 --- ql/src/test/results/clientpositive/auto_join28.q.out +++ ql/src/test/results/clientpositive/auto_join28.q.out @@ -16,31 +16,10 @@ STAGE PLANS: Stage: Stage-6 Map Reduce Local Work Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 src2 Fetch Operator limit: -1 Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - filter predicates: - 0 {(key < 10)} - 1 - 2 {(key < 10)} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 src2 TableScan alias: src2 @@ -50,16 +29,16 @@ STAGE PLANS: type: boolean HashTable Sink Operator condition expressions: - 0 {key} {value} + 0 1 {key} {value} 2 {key} {value} filter predicates: - 0 {(key < 10)} + 0 1 2 {(key < 10)} handleSkewJoin: false keys: - 0 [Column[key]] + 0 1 [Column[key]] 2 [Column[key]] Position of Big Table: 2 @@ -135,6 +114,16 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + src2 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 + src2 Reduce Operator Tree: Extract File Output Operator @@ -480,9 +469,6 @@ STAGE PLANS: src1 Fetch Operator limit: -1 - src2 - Fetch Operator - limit: -1 Alias -> Map Local Operator Tree: src1 TableScan @@ -494,36 +480,18 @@ STAGE PLANS: HashTable Sink Operator condition expressions: 0 {key} {value} - 1 {key} {value} + 1 2 {key} {value} filter predicates: 0 - 1 {(key > 10)} + 1 2 {(key < 10)} handleSkewJoin: false keys: 0 [Column[key]] - 1 [Column[key]] + 1 2 [Column[key]] Position of Big Table: 2 - src2 - TableScan - alias: src2 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - filter predicates: - 0 - 1 {(key > 10)} - 2 {(key < 10)} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 Stage: Stage-2 Map Reduce @@ -596,6 +564,16 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + src2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + src2 + TableScan + alias: src2 Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/auto_join29.q.out ql/src/test/results/clientpositive/auto_join29.q.out index 7798a0e..35dbefb 100644 --- ql/src/test/results/clientpositive/auto_join29.q.out +++ ql/src/test/results/clientpositive/auto_join29.q.out @@ -16,31 +16,10 @@ STAGE PLANS: Stage: Stage-6 Map Reduce Local Work Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 src2 Fetch Operator limit: -1 Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - filter predicates: - 0 {(key < 10)} - 1 - 2 {(key < 10)} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 src2 TableScan alias: src2 @@ -50,16 +29,16 @@ STAGE PLANS: type: boolean HashTable Sink Operator condition expressions: - 0 {key} {value} + 0 1 {key} {value} 2 {key} {value} filter predicates: - 0 {(key < 10)} + 0 1 2 {(key < 10)} handleSkewJoin: false keys: - 0 [Column[key]] + 0 1 [Column[key]] 2 [Column[key]] Position of Big Table: 2 @@ -135,6 +114,16 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + src2 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 + src2 Reduce Operator Tree: Extract File Output Operator @@ -2016,9 +2005,6 @@ STAGE PLANS: src1 Fetch Operator limit: -1 - src2 - Fetch Operator - limit: -1 Alias -> Map Local Operator Tree: src1 TableScan @@ -2030,36 +2016,18 @@ STAGE PLANS: HashTable Sink Operator condition expressions: 0 {key} {value} - 1 {key} {value} + 1 2 {key} {value} filter predicates: 0 - 1 {(key > 10)} + 1 2 {(key < 10)} handleSkewJoin: false keys: 0 [Column[key]] - 1 [Column[key]] + 1 2 [Column[key]] Position of Big Table: 2 - src2 - TableScan - alias: src2 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - filter predicates: - 0 - 1 {(key > 10)} - 2 {(key < 10)} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 Stage: Stage-2 Map Reduce @@ -2132,6 +2100,16 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + src2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + src2 + TableScan + alias: src2 Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/auto_join3.q.out ql/src/test/results/clientpositive/auto_join3.q.out index 4279e8c..a9fc4ce 100644 --- ql/src/test/results/clientpositive/auto_join3.q.out +++ ql/src/test/results/clientpositive/auto_join3.q.out @@ -15,51 +15,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key))) (TOK_TABREF (TOK_TABNAME src) src3) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src3) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src3) value))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-5 depends on stages: Stage-6 + Stage-5 is a root stage Stage-0 depends on stages: Stage-5 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 - src3 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 1 - src3 - TableScan - alias: src3 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -98,6 +58,20 @@ STAGE PLANS: name: default.dest1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + src3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 + src3 + TableScan + alias: src3 Stage: Stage-0 Move Operator diff --git ql/src/test/results/clientpositive/auto_join30.q.out ql/src/test/results/clientpositive/auto_join30.q.out index a46ea2a..d663948 100644 --- ql/src/test/results/clientpositive/auto_join30.q.out +++ ql/src/test/results/clientpositive/auto_join30.q.out @@ -19,12 +19,10 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-7 depends on stages: Stage-1, Stage-4 , consists of Stage-8, Stage-9, Stage-2 - Stage-8 has a backup stage: Stage-2 - Stage-5 depends on stages: Stage-8 + Stage-7 depends on stages: Stage-1, Stage-4 , consists of Stage-5, Stage-6, Stage-2 + Stage-5 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-5, Stage-6 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 + Stage-6 has a backup stage: Stage-2 Stage-2 Stage-4 is a root stage Stage-0 is a root stage @@ -67,25 +65,6 @@ STAGE PLANS: Stage: Stage-7 Conditional Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -125,6 +104,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-3 Map Reduce @@ -157,25 +143,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -215,6 +182,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-2 Map Reduce @@ -354,9 +328,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-6 depends on stages: Stage-1, Stage-4 , consists of Stage-7, Stage-2 - Stage-7 has a backup stage: Stage-2 - Stage-5 depends on stages: Stage-7 + Stage-6 depends on stages: Stage-1, Stage-4 , consists of Stage-5, Stage-2 + Stage-5 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-5 Stage-2 Stage-4 is a root stage @@ -400,25 +373,6 @@ STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -458,6 +412,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-3 Map Reduce @@ -628,9 +589,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-6 depends on stages: Stage-1, Stage-4 , consists of Stage-7, Stage-2 - Stage-7 has a backup stage: Stage-2 - Stage-5 depends on stages: Stage-7 + Stage-6 depends on stages: Stage-1, Stage-4 , consists of Stage-5, Stage-2 + Stage-5 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-5 Stage-2 Stage-4 is a root stage @@ -674,25 +634,6 @@ STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -732,6 +673,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-3 Map Reduce @@ -908,14 +856,11 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-9 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-10, Stage-11, Stage-12, Stage-2 - Stage-10 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-10 + Stage-9 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-6, Stage-7, Stage-8, Stage-2 + Stage-6 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-6, Stage-7, Stage-8 - Stage-11 has a backup stage: Stage-2 - Stage-7 depends on stages: Stage-11 - Stage-12 has a backup stage: Stage-2 - Stage-8 depends on stages: Stage-12 + Stage-7 has a backup stage: Stage-2 + Stage-8 has a backup stage: Stage-2 Stage-2 Stage-4 is a root stage Stage-5 is a root stage @@ -959,43 +904,6 @@ STAGE PLANS: Stage: Stage-9 Conditional Operator - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - $INTNAME2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 0 - $INTNAME2 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -1038,6 +946,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + $INTNAME2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + $INTNAME2 + TableScan Stage: Stage-3 Map Reduce @@ -1070,43 +990,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-11 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 1 - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -1149,43 +1032,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work - - Stage: Stage-12 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - $INTNAME2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 2 - $INTNAME2 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 2 + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + $INTNAME1 + TableScan Stage: Stage-8 Map Reduce @@ -1229,6 +1087,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + $INTNAME2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan + $INTNAME2 + TableScan Stage: Stage-2 Map Reduce @@ -1427,12 +1297,10 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-8 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-9, Stage-10, Stage-2 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 + Stage-8 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-6, Stage-7, Stage-2 + Stage-6 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-6, Stage-7 - Stage-10 has a backup stage: Stage-2 - Stage-7 depends on stages: Stage-10 + Stage-7 has a backup stage: Stage-2 Stage-2 Stage-4 is a root stage Stage-5 is a root stage @@ -1476,43 +1344,6 @@ STAGE PLANS: Stage: Stage-8 Conditional Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - $INTNAME2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 0 - $INTNAME2 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -1555,6 +1386,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + $INTNAME2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + $INTNAME2 + TableScan Stage: Stage-3 Map Reduce @@ -1587,43 +1430,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 1 - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -1666,6 +1472,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + $INTNAME1 + TableScan Stage: Stage-2 Map Reduce @@ -1864,9 +1682,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-7 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-8, Stage-2 - Stage-8 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-8 + Stage-7 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-6, Stage-2 + Stage-6 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-6 Stage-2 Stage-4 is a root stage @@ -1911,43 +1728,6 @@ STAGE PLANS: Stage: Stage-7 Conditional Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - $INTNAME2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 0 - $INTNAME2 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -1990,6 +1770,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + $INTNAME2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + $INTNAME2 + TableScan Stage: Stage-3 Map Reduce @@ -2219,9 +2011,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-7 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-8, Stage-2 - Stage-8 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-8 + Stage-7 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-6, Stage-2 + Stage-6 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-6 Stage-2 Stage-4 is a root stage @@ -2266,43 +2057,6 @@ STAGE PLANS: Stage: Stage-7 Conditional Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - $INTNAME2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 2 - $INTNAME2 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 2 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -2345,6 +2099,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + $INTNAME2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan + $INTNAME2 + TableScan Stage: Stage-3 Map Reduce @@ -2574,9 +2340,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-7 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-8, Stage-2 - Stage-8 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-8 + Stage-7 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-6, Stage-2 + Stage-6 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-6 Stage-2 Stage-4 is a root stage @@ -2621,43 +2386,6 @@ STAGE PLANS: Stage: Stage-7 Conditional Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - $INTNAME2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 2 - $INTNAME2 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 2 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -2700,6 +2428,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + $INTNAME2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan + $INTNAME2 + TableScan Stage: Stage-3 Map Reduce diff --git ql/src/test/results/clientpositive/auto_join31.q.out ql/src/test/results/clientpositive/auto_join31.q.out index 4504426..87c9f75 100644 --- ql/src/test/results/clientpositive/auto_join31.q.out +++ ql/src/test/results/clientpositive/auto_join31.q.out @@ -25,12 +25,10 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-8 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-9, Stage-10, Stage-2 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 + Stage-8 depends on stages: Stage-1, Stage-4, Stage-5 , consists of Stage-6, Stage-7, Stage-2 + Stage-6 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-6, Stage-7 - Stage-10 has a backup stage: Stage-2 - Stage-7 depends on stages: Stage-10 + Stage-7 has a backup stage: Stage-2 Stage-2 Stage-4 is a root stage Stage-5 is a root stage @@ -74,43 +72,6 @@ STAGE PLANS: Stage: Stage-8 Conditional Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 1 - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -153,6 +114,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + $INTNAME1 + TableScan Stage: Stage-3 Map Reduce @@ -185,43 +158,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - $INTNAME2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 2 - $INTNAME2 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {_col0} {_col1} - 2 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - 2 [Column[_col0]] - Position of Big Table: 2 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -264,6 +200,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + $INTNAME2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan + $INTNAME2 + TableScan Stage: Stage-2 Map Reduce diff --git ql/src/test/results/clientpositive/auto_join32.q.out ql/src/test/results/clientpositive/auto_join32.q.out index da4ef55..653cee4 100644 --- ql/src/test/results/clientpositive/auto_join32.q.out +++ ql/src/test/results/clientpositive/auto_join32.q.out @@ -24,31 +24,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME studenttab10k) s) (TOK_TABREF (TOK_TABNAME votertab10k) v) (= (. (TOK_TABLE_OR_COL s) name) (. (TOK_TABLE_OR_COL v) name)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL s) name)) (TOK_SELEXPR (TOK_FUNCTIONDI count (TOK_TABLE_OR_COL registration)))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL s) name)))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-2 depends on stages: Stage-5 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - s - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - s - TableScan - alias: s - HashTable Sink Operator - condition expressions: - 0 {name} - 1 {registration} - handleSkewJoin: false - keys: - 0 [Column[name]] - 1 [Column[name]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -101,6 +80,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + s + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + s + TableScan + alias: s Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/auto_join9.q.out ql/src/test/results/clientpositive/auto_join9.q.out index d2589ed..d8fc94c 100644 --- ql/src/test/results/clientpositive/auto_join9.q.out +++ ql/src/test/results/clientpositive/auto_join9.q.out @@ -15,32 +15,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcpart) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) value))) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL src1) ds) '2008-04-08') (= (. (TOK_TABLE_OR_COL src1) hr) '12'))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-4 depends on stages: Stage-5 + Stage-4 is a root stage Stage-0 depends on stages: Stage-4 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -76,6 +55,14 @@ STAGE PLANS: name: default.dest1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 Stage: Stage-0 Move Operator diff --git ql/src/test/results/clientpositive/auto_join_without_localtask.q.out ql/src/test/results/clientpositive/auto_join_without_localtask.q.out new file mode 100644 index 0000000..ae3a4ed --- /dev/null +++ ql/src/test/results/clientpositive/auto_join_without_localtask.q.out @@ -0,0 +1,1025 @@ +PREHOOK: query: explain +select a.* from src a join src b on a.key=b.key limit 40 +PREHOOK: type: QUERY +POSTHOOK: query: explain +select a.* from src a join src b on a.key=b.key limit 40 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_TABREF (TOK_TABNAME src) 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_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME a)))) (TOK_LIMIT 40))) + +STAGE DEPENDENCIES: + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 + Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-5 + Conditional Operator + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + b + TableScan + alias: b + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 40 + +PREHOOK: query: select a.* from src a join src b on a.key=b.key limit 40 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select a.* from src a join src b on a.key=b.key limit 40 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +PREHOOK: query: explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40 +PREHOOK: type: QUERY +POSTHOOK: query: explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_TABREF (TOK_TABNAME src) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF (TOK_TABNAME src) c) (= (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL c) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME a)))) (TOK_LIMIT 40))) + +STAGE DEPENDENCIES: + Stage-10 is a root stage , consists of Stage-8, Stage-9, Stage-1 + Stage-8 has a backup stage: Stage-1 + Stage-7 depends on stages: Stage-1, Stage-8, Stage-9 , consists of Stage-5, Stage-6, Stage-2 + Stage-5 has a backup stage: Stage-2 + Stage-6 has a backup stage: Stage-2 + Stage-2 + Stage-9 has a backup stage: Stage-1 + Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-10 + Conditional Operator + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {_col0} {_col1} + 1 + handleSkewJoin: false + keys: + 0 [Column[_col1]] + 1 [Column[value]] + outputColumnNames: _col4, _col5 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col4 + type: string + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + c + TableScan + alias: c + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {_col0} {_col1} + 1 + handleSkewJoin: false + keys: + 0 [Column[_col1]] + 1 [Column[value]] + outputColumnNames: _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col4 + type: string + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + TableScan + Reduce Output Operator + key expressions: + expr: _col1 + type: string + sort order: + + Map-reduce partition columns: + expr: _col1 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + c + TableScan + alias: c + Reduce Output Operator + key expressions: + expr: value + type: string + sort order: + + Map-reduce partition columns: + expr: value + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col4} {VALUE._col5} + 1 + handleSkewJoin: false + outputColumnNames: _col4, _col5 + Select Operator + expressions: + expr: _col4 + type: string + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-9 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + b + TableScan + alias: b + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + + Stage: Stage-0 + Fetch Operator + limit: 40 + +PREHOOK: query: select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +238 val_238 +238 val_238 +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +PREHOOK: query: explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40 +PREHOOK: type: QUERY +POSTHOOK: query: explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_TABREF (TOK_TABNAME src) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF (TOK_TABNAME src) c) (= (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL c) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME a)))) (TOK_WHERE (> (. (TOK_TABLE_OR_COL a) key) 100)) (TOK_LIMIT 40))) + +STAGE DEPENDENCIES: + Stage-10 is a root stage , consists of Stage-13, Stage-14, Stage-1 + Stage-13 has a backup stage: Stage-1 + Stage-8 depends on stages: Stage-13 + Stage-7 depends on stages: Stage-1, Stage-8, Stage-9 , consists of Stage-5, Stage-6, Stage-2 + Stage-5 has a backup stage: Stage-2 + Stage-6 has a backup stage: Stage-2 + Stage-2 + Stage-14 has a backup stage: Stage-1 + Stage-9 depends on stages: Stage-14 + Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-10 + Conditional Operator + + Stage: Stage-13 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: + expr: (key > 100) + type: boolean + HashTable Sink Operator + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: + expr: (key > 100) + type: boolean + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {_col0} {_col1} + 1 + handleSkewJoin: false + keys: + 0 [Column[_col1]] + 1 [Column[value]] + outputColumnNames: _col4, _col5 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col4 + type: string + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + c + TableScan + alias: c + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {_col0} {_col1} + 1 + handleSkewJoin: false + keys: + 0 [Column[_col1]] + 1 [Column[value]] + outputColumnNames: _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col4 + type: string + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + TableScan + Reduce Output Operator + key expressions: + expr: _col1 + type: string + sort order: + + Map-reduce partition columns: + expr: _col1 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + c + TableScan + alias: c + Reduce Output Operator + key expressions: + expr: value + type: string + sort order: + + Map-reduce partition columns: + expr: value + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col4} {VALUE._col5} + 1 + handleSkewJoin: false + outputColumnNames: _col4, _col5 + Select Operator + expressions: + expr: _col4 + type: string + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + Limit + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-14 + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: + expr: (key > 100) + type: boolean + HashTable Sink Operator + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + + Stage: Stage-9 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: + expr: (key > 100) + type: boolean + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: + expr: (key > 100) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + b + TableScan + alias: b + Filter Operator + predicate: + expr: (key > 100) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + + Stage: Stage-0 + Fetch Operator + limit: 40 + +PREHOOK: query: select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +238 val_238 +238 val_238 +238 val_238 +238 val_238 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +484 val_484 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +193 val_193 +PREHOOK: query: -- fallback to common join +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +Execution failed with exit status: 3 +Obtaining error information + +Task failed! +Task ID: + Stage-14 + +Logs: + +#### A masked pattern was here #### +FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-5 + +Logs: + +#### A masked pattern was here #### +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask +POSTHOOK: query: -- fallback to common join +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +119 val_119 diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out index 8e11616..2f03c0b 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out @@ -475,11 +475,9 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_big) a) (TOK_TABREF (TOK_TABNAME bucket_small) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -487,70 +485,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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 string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - 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-3 Map Reduce Alias -> Map Operator Tree: @@ -584,6 +518,58 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### 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 string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -755,114 +741,6 @@ STAGE PLANS: GatherStats: false MultiFileSpray: false - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -896,6 +774,102 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out index 95e86e1..2a9eb54 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out @@ -83,77 +83,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-2 depends on stages: Stage-5 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 1 dataSize: 114 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -197,6 +130,60 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 1 dataSize: 114 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -409,77 +396,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-2 depends on stages: Stage-5 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 1 dataSize: 114 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -523,6 +443,60 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 1 dataSize: 114 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -729,83 +703,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 1 dataSize: 114 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/smallsrcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/smallsrcsortbucket2outof4.txt], ds=2008-04-08/srcsortbucket3outof4.txt=[ds=2008-04-08/smallsrcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket4outof4.txt=[ds=2008-04-08/smallsrcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/smallsrcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/smallsrcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket3outof4.txt=[ds=2008-04-08/smallsrcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket4outof4.txt=[ds=2008-04-08/smallsrcsortbucket2outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -850,6 +751,66 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 1 dataSize: 114 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/smallsrcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/smallsrcsortbucket2outof4.txt], ds=2008-04-08/srcsortbucket3outof4.txt=[ds=2008-04-08/smallsrcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket4outof4.txt=[ds=2008-04-08/smallsrcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/smallsrcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/smallsrcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket3outof4.txt=[ds=2008-04-08/smallsrcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket4outof4.txt=[ds=2008-04-08/smallsrcsortbucket2outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out index fd8e9cc..05d2236 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out @@ -111,206 +111,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_medium) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF (TOK_TABNAME bucket_big) c) (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF (TOK_TABNAME bucket_medium) d) (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-8 is a root stage - Stage-2 depends on stages: Stage-8 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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 string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - b - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 3 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_medium - numFiles 3 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_medium { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 170 -#### 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 3 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_medium - partition_columns ds - serialization.ddl struct bucket_medium { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_medium - name: default.bucket_medium - d - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 3 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_medium - numFiles 3 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_medium { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 170 -#### 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 3 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_medium - partition_columns ds - serialization.ddl struct bucket_medium { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_medium - name: default.bucket_medium - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 1 dataSize: 114 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - b - TableScan - alias: b - Statistics: - numRows: 1 dataSize: 170 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - d - TableScan - alias: d - Statistics: - numRows: 0 dataSize: 170 basicStatsState: PARTIAL colStatsState: COMPLETE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [] - 1 [] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -370,6 +174,167 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### 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 string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + b + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 3 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_medium + numFiles 3 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_medium { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 170 +#### 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 3 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_medium + partition_columns ds + serialization.ddl struct bucket_medium { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_medium + name: default.bucket_medium + d + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 3 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_medium + numFiles 3 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_medium { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 170 +#### 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 3 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_medium + partition_columns ds + serialization.ddl struct bucket_medium { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_medium + name: default.bucket_medium + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 1 dataSize: 114 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + b + TableScan + alias: b + Statistics: + numRows: 1 dataSize: 170 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + d + TableScan + alias: d + Statistics: + numRows: 0 dataSize: 170 basicStatsState: PARTIAL colStatsState: COMPLETE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_14.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_14.q.out index 3c20e0f..8652df9 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_14.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_14.q.out @@ -46,9 +46,8 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-4 is a root stage , consists of Stage-5, Stage-1 - Stage-5 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-5 + Stage-4 is a root stage , consists of Stage-3, Stage-1 + Stage-3 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -56,26 +55,6 @@ STAGE PLANS: Stage: Stage-4 Conditional Operator - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -108,6 +87,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: @@ -229,9 +216,8 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-4 is a root stage , consists of Stage-5, Stage-1 - Stage-5 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-5 + Stage-4 is a root stage , consists of Stage-3, Stage-1 + Stage-3 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -239,26 +225,6 @@ STAGE PLANS: Stage: Stage-4 Conditional Operator - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -291,6 +257,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_15.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_15.q.out index 6189df6..199a5d9 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_15.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_15.q.out @@ -44,9 +44,8 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-4 is a root stage , consists of Stage-5, Stage-1 - Stage-5 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-5 + Stage-4 is a root stage , consists of Stage-3, Stage-1 + Stage-3 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -54,26 +53,6 @@ STAGE PLANS: Stage: Stage-4 Conditional Operator - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -106,6 +85,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: @@ -194,9 +181,8 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-4 is a root stage , consists of Stage-5, Stage-1 - Stage-5 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-5 + Stage-4 is a root stage , consists of Stage-3, Stage-1 + Stage-3 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -204,26 +190,6 @@ STAGE PLANS: Stage: Stage-4 Conditional Operator - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -256,6 +222,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out index c00b519..4dd6181 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out @@ -267,11 +267,9 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_big) a) (TOK_TABREF (TOK_TABNAME bucket_small) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -279,70 +277,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 226 -#### 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - 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-3 Map Reduce Alias -> Map Operator Tree: @@ -376,6 +310,58 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 226 +#### 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -547,114 +533,6 @@ STAGE PLANS: GatherStats: false MultiFileSpray: false - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -688,6 +566,102 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1031,6 +1005,18 @@ PREHOOK: Input: default@bucket_big@ds=2008-04-09 PREHOOK: Input: default@bucket_small PREHOOK: Input: default@bucket_small@ds=2008-04-08 #### A masked pattern was here #### +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-3 + +Logs: + +#### A masked pattern was here #### +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key POSTHOOK: type: QUERY POSTHOOK: Input: default@bucket_big diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out index 42bab30..f718548 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out @@ -369,11 +369,9 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_big) a) (TOK_TABREF (TOK_TABNAME bucket_small) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -381,112 +379,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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 string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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 string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - 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-3 Map Reduce Alias -> Map Operator Tree: @@ -520,6 +412,100 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### 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 string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### 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 string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -689,71 +675,6 @@ STAGE PLANS: GatherStats: false MultiFileSpray: false - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -787,6 +708,59 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out index ec33acd..036323f 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out @@ -381,11 +381,9 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_big) a) (TOK_TABREF (TOK_TABNAME bucket_small) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -393,112 +391,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 226 -#### 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 226 -#### 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - 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-3 Map Reduce Alias -> Map Operator Tree: @@ -532,6 +424,100 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 226 +#### 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 226 +#### 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -701,71 +687,6 @@ STAGE PLANS: GatherStats: false MultiFileSpray: false - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -799,6 +720,59 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out index f8c164a..6b2e719 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out @@ -346,11 +346,9 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_big) a) (TOK_TABREF (TOK_TABNAME bucket_small) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -358,27 +356,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - 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-3 Map Reduce Alias -> Map Operator Tree: @@ -412,6 +389,15 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -535,27 +521,6 @@ STAGE PLANS: GatherStats: false MultiFileSpray: false - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -589,6 +554,15 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_6.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_6.q.out index 6ee49a7..0d1257f 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_6.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_6.q.out @@ -531,14 +531,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF (TOK_TABNAME src) c) (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL a) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-8 is a root stage , consists of Stage-9, Stage-10, Stage-11, Stage-1 - Stage-9 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-9 + Stage-8 is a root stage , consists of Stage-5, Stage-6, Stage-7, Stage-1 + Stage-5 has a backup stage: Stage-1 Stage-2 depends on stages: Stage-1, Stage-5, Stage-6, Stage-7 - Stage-10 has a backup stage: Stage-1 - Stage-6 depends on stages: Stage-10 - Stage-11 has a backup stage: Stage-1 - Stage-7 depends on stages: Stage-11 + Stage-6 has a backup stage: Stage-1 + Stage-7 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -546,45 +543,6 @@ STAGE PLANS: Stage: Stage-8 Conditional Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 0 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -621,6 +579,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + c + TableScan + alias: c Stage: Stage-2 Map Reduce @@ -653,45 +625,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 1 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 1 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -728,45 +661,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work - - Stage: Stage-11 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 2 - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 2 + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + c + TableScan + alias: c Stage: Stage-7 Map Reduce @@ -804,6 +712,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + b + TableScan + alias: b Stage: Stage-1 Map Reduce @@ -1294,14 +1216,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF (TOK_TABNAME src) c) (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL a) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-8 is a root stage , consists of Stage-9, Stage-10, Stage-11, Stage-1 - Stage-9 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-9 + Stage-8 is a root stage , consists of Stage-5, Stage-6, Stage-7, Stage-1 + Stage-5 has a backup stage: Stage-1 Stage-2 depends on stages: Stage-1, Stage-5, Stage-6, Stage-7 - Stage-10 has a backup stage: Stage-1 - Stage-6 depends on stages: Stage-10 - Stage-11 has a backup stage: Stage-1 - Stage-7 depends on stages: Stage-11 + Stage-6 has a backup stage: Stage-1 + Stage-7 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -1309,45 +1228,6 @@ STAGE PLANS: Stage: Stage-8 Conditional Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 0 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -1384,6 +1264,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + c + TableScan + alias: c Stage: Stage-2 Map Reduce @@ -1416,45 +1310,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 1 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 1 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -1491,45 +1346,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work - - Stage: Stage-11 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 2 - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 2 + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + c + TableScan + alias: c Stage: Stage-7 Map Reduce @@ -1567,6 +1397,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + b + TableScan + alias: b Stage: Stage-1 Map Reduce @@ -1675,13 +1519,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF (TOK_TABNAME tbl3) c) (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL a) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-7 is a root stage , consists of Stage-8, Stage-9, Stage-10, Stage-1 - Stage-8 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-8 - Stage-9 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-9 - Stage-10 has a backup stage: Stage-1 - Stage-6 depends on stages: Stage-10 + Stage-7 is a root stage , consists of Stage-4, Stage-5, Stage-6, Stage-1 + Stage-4 has a backup stage: Stage-1 + Stage-5 has a backup stage: Stage-1 + Stage-6 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -1689,45 +1530,6 @@ STAGE PLANS: Stage: Stage-7 Conditional Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 0 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -1763,6 +1565,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + c + TableScan + alias: c Reduce Operator Tree: Group By Operator aggregations: @@ -1783,45 +1599,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 1 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -1857,6 +1634,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + c + TableScan + alias: c Reduce Operator Tree: Group By Operator aggregations: @@ -1877,45 +1668,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -1951,6 +1703,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out index 3f2672e..d6c3dc6 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out @@ -488,11 +488,9 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_big) a) (TOK_TABREF (TOK_TABNAME bucket_small) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -500,112 +498,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 226 -#### 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 226 -#### 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - 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-3 Map Reduce Alias -> Map Operator Tree: @@ -639,6 +531,100 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 226 +#### 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 226 +#### 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -853,114 +839,6 @@ STAGE PLANS: GatherStats: false MultiFileSpray: false - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -994,6 +872,102 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out index 4cfd022..153e223 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out @@ -490,11 +490,9 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_big) a) (TOK_TABREF (TOK_TABNAME bucket_small) 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_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -502,112 +500,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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 string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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 string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - 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-3 Map Reduce Alias -> Map Operator Tree: @@ -641,6 +533,100 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### 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 string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### 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 string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -855,114 +841,6 @@ STAGE PLANS: GatherStats: false MultiFileSpray: false - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_big { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -996,6 +874,102 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_big { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1384,6 +1358,18 @@ PREHOOK: Input: default@bucket_small PREHOOK: Input: default@bucket_small@ds=2008-04-08 PREHOOK: Input: default@bucket_small@ds=2008-04-09 #### A masked pattern was here #### +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-3 + +Logs: + +#### A masked pattern was here #### +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key POSTHOOK: type: QUERY POSTHOOK: Input: default@bucket_big diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_9.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_9.q.out index 7b9f811..7295490 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_9.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_9.q.out @@ -510,12 +510,10 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-9 depends on stages: Stage-1, Stage-5 , consists of Stage-10, Stage-11, Stage-3 - Stage-10 has a backup stage: Stage-3 - Stage-7 depends on stages: Stage-10 + Stage-9 depends on stages: Stage-1, Stage-5 , consists of Stage-7, Stage-8, Stage-3 + Stage-7 has a backup stage: Stage-3 Stage-4 depends on stages: Stage-3, Stage-7, Stage-8 - Stage-11 has a backup stage: Stage-3 - Stage-8 depends on stages: Stage-11 + Stage-8 has a backup stage: Stage-3 Stage-3 Stage-5 is a root stage Stage-0 is a root stage @@ -593,25 +591,6 @@ STAGE PLANS: Stage: Stage-9 Conditional Operator - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -647,6 +626,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-4 Map Reduce @@ -680,25 +666,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-11 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-8 Map Reduce Alias -> Map Operator Tree: @@ -734,6 +701,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-3 Map Reduce @@ -2091,11 +2065,9 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) val1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) val2)))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-5 is a root stage , consists of Stage-3, Stage-4, Stage-1 + Stage-3 has a backup stage: Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -2103,26 +2075,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - subq1:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq1:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -2155,6 +2107,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq1:b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: @@ -2175,26 +2135,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - subq1:a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq1:a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -2227,6 +2167,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq1:a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: @@ -2348,12 +2296,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) val1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) val2)))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_GROUPBY (TOK_TABLE_OR_COL key)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) STAGE DEPENDENCIES: - Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-6 is a root stage , consists of Stage-4, Stage-5, Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-2 depends on stages: Stage-1, Stage-4, Stage-5 - Stage-8 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-8 + Stage-5 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -2361,26 +2307,6 @@ STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - subq1:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq1:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -2427,6 +2353,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq1:b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: @@ -2478,26 +2412,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - subq1:a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq1:a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -2544,6 +2458,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq1:a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: @@ -2702,12 +2624,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) val1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) val2)))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) subq2)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-6 is a root stage , consists of Stage-4, Stage-5, Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-3 depends on stages: Stage-1, Stage-4, Stage-5 - Stage-8 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-8 + Stage-5 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -2715,26 +2635,6 @@ STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - subq2:subq1:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq2:subq1:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -2781,6 +2681,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq2:subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq2:subq1:b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: @@ -2837,26 +2745,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - subq2:subq1:a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq2:subq1:a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -2903,6 +2791,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq2:subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq2:subq1:a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: @@ -3078,24 +2974,18 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) val1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) val2)))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt1)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) src1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_TABREF (TOK_TABNAME tbl2) 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_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) val1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) val2)))) subq2)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt1)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) cnt1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) cnt1))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src1) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src1) cnt1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src2) cnt1))))) STAGE DEPENDENCIES: - Stage-12 is a root stage , consists of Stage-18, Stage-19, Stage-1 - Stage-18 has a backup stage: Stage-1 - Stage-10 depends on stages: Stage-18 - Stage-9 depends on stages: Stage-1, Stage-5, Stage-10, Stage-11, Stage-13, Stage-14 , consists of Stage-16, Stage-17, Stage-3 - Stage-16 has a backup stage: Stage-3 - Stage-7 depends on stages: Stage-16 + Stage-12 is a root stage , consists of Stage-10, Stage-11, Stage-1 + Stage-10 has a backup stage: Stage-1 + Stage-9 depends on stages: Stage-1, Stage-5, Stage-10, Stage-11, Stage-13, Stage-14 , consists of Stage-7, Stage-8, Stage-3 + Stage-7 has a backup stage: Stage-3 Stage-4 depends on stages: Stage-3, Stage-7, Stage-8 - Stage-17 has a backup stage: Stage-3 - Stage-8 depends on stages: Stage-17 + Stage-8 has a backup stage: Stage-3 Stage-3 - Stage-19 has a backup stage: Stage-1 - Stage-11 depends on stages: Stage-19 + Stage-11 has a backup stage: Stage-1 Stage-1 - Stage-15 is a root stage , consists of Stage-20, Stage-21, Stage-5 - Stage-20 has a backup stage: Stage-5 - Stage-13 depends on stages: Stage-20 - Stage-21 has a backup stage: Stage-5 - Stage-14 depends on stages: Stage-21 + Stage-15 is a root stage , consists of Stage-13, Stage-14, Stage-5 + Stage-13 has a backup stage: Stage-5 + Stage-14 has a backup stage: Stage-5 Stage-5 Stage-0 is a root stage @@ -3103,26 +2993,6 @@ STAGE PLANS: Stage: Stage-12 Conditional Operator - Stage: Stage-18 - Map Reduce Local Work - Alias -> Map Local Tables: - src1:subq1:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1:subq1:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-10 Map Reduce Alias -> Map Operator Tree: @@ -3169,6 +3039,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1:subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1:subq1:b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: @@ -3197,25 +3075,6 @@ STAGE PLANS: Stage: Stage-9 Conditional Operator - Stage: Stage-16 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -3251,6 +3110,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-4 Map Reduce @@ -3284,25 +3150,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-17 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-8 Map Reduce Alias -> Map Operator Tree: @@ -3338,6 +3185,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-3 Map Reduce @@ -3398,26 +3252,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-19 - Map Reduce Local Work - Alias -> Map Local Tables: - src1:subq1:a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1:subq1:a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-11 Map Reduce Alias -> Map Operator Tree: @@ -3464,6 +3298,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1:subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1:subq1:a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: @@ -3561,26 +3403,6 @@ STAGE PLANS: Stage: Stage-15 Conditional Operator - Stage: Stage-20 - Map Reduce Local Work - Alias -> Map Local Tables: - src2:subq2:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src2:subq2:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-13 Map Reduce Alias -> Map Operator Tree: @@ -3627,6 +3449,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src2:subq2:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src2:subq2:b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: @@ -3652,26 +3482,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-21 - Map Reduce Local Work - Alias -> Map Local Tables: - src2:subq2:a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src2:subq2:a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-14 Map Reduce Alias -> Map Operator Tree: @@ -3718,6 +3528,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src2:subq2:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src2:subq2:a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: @@ -4206,9 +4024,8 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tbl1) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) value)) (TOK_WHERE (< (TOK_TABLE_OR_COL key) 8)))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_WHERE (< (TOK_TABLE_OR_COL key) 6)))) subq2) (TOK_TABREF (TOK_TABNAME tbl2) b) (= (. (TOK_TABLE_OR_COL subq2) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 + Stage-5 is a root stage , consists of Stage-3, Stage-7, Stage-1 + Stage-3 has a backup stage: Stage-1 Stage-7 has a backup stage: Stage-1 Stage-4 depends on stages: Stage-7 Stage-1 @@ -4218,26 +4035,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -4279,6 +4076,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: @@ -5154,9 +4959,8 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tbl1) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) value)) (TOK_WHERE (< (TOK_TABLE_OR_COL key) 6)))) subq1) (TOK_TABREF (TOK_TABNAME tbl2) a) (= (. (TOK_TABLE_OR_COL subq1) key) (. (TOK_TABLE_OR_COL a) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 + Stage-5 is a root stage , consists of Stage-3, Stage-7, Stage-1 + Stage-3 has a backup stage: Stage-1 Stage-7 has a backup stage: Stage-1 Stage-4 depends on stages: Stage-7 Stage-1 @@ -5166,26 +4970,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -5227,6 +5011,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: @@ -5434,11 +5226,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME tbl1) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tbl2) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) value)) (TOK_WHERE (< (TOK_TABLE_OR_COL key) 6)))) subq1) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL subq1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 + Stage-5 is a root stage , consists of Stage-6, Stage-4, Stage-1 Stage-6 has a backup stage: Stage-1 Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-4 has a backup stage: Stage-1 Stage-1 Stage-0 is a root stage @@ -5527,26 +5318,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -5588,6 +5359,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Group By Operator aggregations: @@ -6225,9 +6004,8 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tbl1) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value) value)) (TOK_WHERE (< (TOK_TABLE_OR_COL key) 8)))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_WHERE (< (TOK_TABLE_OR_COL key) 6)))) subq2) (TOK_TABREF (TOK_TABNAME tbl2) b) (= (. (TOK_TABLE_OR_COL subq2) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq2) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq2) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) value2)))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 + Stage-5 is a root stage , consists of Stage-3, Stage-7, Stage-1 + Stage-3 has a backup stage: Stage-1 Stage-7 has a backup stage: Stage-1 Stage-4 depends on stages: Stage-7 Stage-1 @@ -6237,26 +6015,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - a:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -6298,6 +6056,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:b + TableScan + alias: b Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/bucket_map_join_1.q.out ql/src/test/results/clientpositive/bucket_map_join_1.q.out index f052025..7e945d0 100644 --- ql/src/test/results/clientpositive/bucket_map_join_1.q.out +++ ql/src/test/results/clientpositive/bucket_map_join_1.q.out @@ -50,34 +50,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME table1) a) (TOK_TABREF (TOK_TABNAME table2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL b) value))))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 0 dataSize: 21 basicStatsState: PARTIAL colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -121,6 +97,17 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 0 dataSize: 21 basicStatsState: PARTIAL colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucket_map_join_2.q.out ql/src/test/results/clientpositive/bucket_map_join_2.q.out index eb51e23..9420338 100644 --- ql/src/test/results/clientpositive/bucket_map_join_2.q.out +++ ql/src/test/results/clientpositive/bucket_map_join_2.q.out @@ -50,34 +50,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME table1) a) (TOK_TABREF (TOK_TABNAME table2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL b) value))))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 0 dataSize: 21 basicStatsState: PARTIAL colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -121,6 +97,17 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 0 dataSize: 21 basicStatsState: PARTIAL colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketcontext_1.q.out ql/src/test/results/clientpositive/bucketcontext_1.q.out index 9bdc1ee..6d5fd24 100644 --- ql/src/test/results/clientpositive/bucketcontext_1.q.out +++ ql/src/test/results/clientpositive/bucketcontext_1.q.out @@ -81,84 +81,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 27 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt], ds=2008-04-08/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -203,6 +129,67 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 27 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt], ds=2008-04-08/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketcontext_2.q.out ql/src/test/results/clientpositive/bucketcontext_2.q.out index d5d67dc..82d0d7d 100644 --- ql/src/test/results/clientpositive/bucketcontext_2.q.out +++ ql/src/test/results/clientpositive/bucketcontext_2.q.out @@ -69,84 +69,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -191,6 +117,67 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketcontext_3.q.out ql/src/test/results/clientpositive/bucketcontext_3.q.out index b226246..4e8fb6e 100644 --- ql/src/test/results/clientpositive/bucketcontext_3.q.out +++ ql/src/test/results/clientpositive/bucketcontext_3.q.out @@ -69,126 +69,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 54 dataSize: 5500 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt], ds=2008-04-08/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -233,6 +117,109 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 54 dataSize: 5500 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt], ds=2008-04-08/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketcontext_4.q.out ql/src/test/results/clientpositive/bucketcontext_4.q.out index c27d927..d25a50a 100644 --- ql/src/test/results/clientpositive/bucketcontext_4.q.out +++ ql/src/test/results/clientpositive/bucketcontext_4.q.out @@ -81,126 +81,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 116 dataSize: 11624 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket3outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket4outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -245,6 +129,109 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 116 dataSize: 11624 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket3outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket4outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketcontext_5.q.out ql/src/test/results/clientpositive/bucketcontext_5.q.out index 819362f..cabef60 100644 --- ql/src/test/results/clientpositive/bucketcontext_5.q.out +++ ql/src/test/results/clientpositive/bucketcontext_5.q.out @@ -54,41 +54,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {srcsortbucket1outof4.txt=[srcsortbucket1outof4.txt, srcsortbucket3outof4.txt], srcsortbucket2outof4.txt=[srcsortbucket2outof4.txt, srcsortbucket4outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -133,6 +102,24 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {srcsortbucket1outof4.txt=[srcsortbucket1outof4.txt, srcsortbucket3outof4.txt], srcsortbucket2outof4.txt=[srcsortbucket2outof4.txt, srcsortbucket4outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketcontext_6.q.out ql/src/test/results/clientpositive/bucketcontext_6.q.out index d7b1ad7..12a0151 100644 --- ql/src/test/results/clientpositive/bucketcontext_6.q.out +++ ql/src/test/results/clientpositive/bucketcontext_6.q.out @@ -68,41 +68,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcsortbucket1outof4.txt=[srcsortbucket1outof4.txt, srcsortbucket3outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[srcsortbucket2outof4.txt, srcsortbucket4outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[srcsortbucket1outof4.txt, srcsortbucket3outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[srcsortbucket2outof4.txt, srcsortbucket4outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -147,6 +116,24 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcsortbucket1outof4.txt=[srcsortbucket1outof4.txt, srcsortbucket3outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[srcsortbucket2outof4.txt, srcsortbucket4outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[srcsortbucket1outof4.txt, srcsortbucket3outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[srcsortbucket2outof4.txt, srcsortbucket4outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketcontext_7.q.out ql/src/test/results/clientpositive/bucketcontext_7.q.out index 80749f0..eeec5ed 100644 --- ql/src/test/results/clientpositive/bucketcontext_7.q.out +++ ql/src/test/results/clientpositive/bucketcontext_7.q.out @@ -94,126 +94,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string 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 4 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 116 dataSize: 11624 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket3outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket4outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket3outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket4outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -258,6 +142,109 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string 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 4 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 116 dataSize: 11624 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket3outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket4outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-08/srcsortbucket3outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket3outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-08/srcsortbucket4outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket4outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketcontext_8.q.out ql/src/test/results/clientpositive/bucketcontext_8.q.out index b8a4d4f..913d6e6 100644 --- ql/src/test/results/clientpositive/bucketcontext_8.q.out +++ ql/src/test/results/clientpositive/bucketcontext_8.q.out @@ -94,126 +94,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME bucket_big) 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 a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 54 dataSize: 5500 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt], ds=2008-04-08/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -258,6 +142,109 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 54 dataSize: 5500 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt], ds=2008-04-08/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-08/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket1outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket2outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt], ds=2008-04-09/srcsortbucket3outof4.txt=[ds=2008-04-08/srcsortbucket1outof4.txt, ds=2008-04-09/srcsortbucket1outof4.txt], ds=2008-04-09/srcsortbucket4outof4.txt=[ds=2008-04-08/srcsortbucket2outof4.txt, ds=2008-04-09/srcsortbucket2outof4.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin1.q.out ql/src/test/results/clientpositive/bucketmapjoin1.q.out index 8d24d6f..a4f480c 100644 --- ql/src/test/results/clientpositive/bucketmapjoin1.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin1.q.out @@ -161,39 +161,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) 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 a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {} - Alias Bucket File Name Mapping: - a {} - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -260,6 +231,22 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {} + Alias Bucket File Name Mapping: + a {} Stage: Stage-0 Fetch Operator @@ -360,8 +347,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-8 depends on stages: Stage-1 , consists of Stage-5, Stage-4, Stage-6 Stage-5 Stage-0 depends on stages: Stage-5, Stage-4, Stage-7 @@ -371,78 +357,6 @@ STAGE DEPENDENCIES: Stage-7 depends on stages: Stage-6 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part { 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: - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part - name: default.srcbucket_mapjoin_part - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 55 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {srcbucket20.txt=[ds=2008-04-08/srcbucket20.txt, ds=2008-04-08/srcbucket22.txt], srcbucket21.txt=[ds=2008-04-08/srcbucket21.txt, ds=2008-04-08/srcbucket23.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -506,6 +420,66 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part { 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: + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part + name: default.srcbucket_mapjoin_part + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 55 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {srcbucket20.txt=[ds=2008-04-08/srcbucket20.txt, ds=2008-04-08/srcbucket22.txt], srcbucket21.txt=[ds=2008-04-08/srcbucket21.txt, ds=2008-04-08/srcbucket23.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -901,8 +875,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -912,36 +885,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcbucket20.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket21.txt=[srcbucket21.txt], ds=2008-04-08/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket23.txt=[srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -1010,6 +953,24 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcbucket20.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket21.txt=[srcbucket21.txt], ds=2008-04-08/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket23.txt=[srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin10.q.out ql/src/test/results/clientpositive/bucketmapjoin10.q.out index 371af7d..561465f 100644 --- ql/src/test/results/clientpositive/bucketmapjoin10.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin10.q.out @@ -118,117 +118,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (AND (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL a) part))) (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL b) part))))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 3 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 3 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 4200 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 3 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 2 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 3 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 1737 dataSize: 6950 basicStatsState: COMPLETE colStatsState: NONE - 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: @@ -272,6 +165,100 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 3 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 3 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 4200 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 3 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 2 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 3 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 1737 dataSize: 6950 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin11.q.out ql/src/test/results/clientpositive/bucketmapjoin11.q.out index fcfa41f..51c3927 100644 --- ql/src/test/results/clientpositive/bucketmapjoin11.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin11.q.out @@ -124,124 +124,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (AND (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL a) part))) (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL b) part))))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 4 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 2 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 2140 dataSize: 8562 basicStatsState: COMPLETE colStatsState: NONE - 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 {part=1/srcbucket20.txt=[part=1/srcbucket20.txt, part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt, part=1/srcbucket23.txt, part=2/srcbucket21.txt], part=2/srcbucket20.txt=[part=1/srcbucket20.txt, part=2/srcbucket20.txt], part=2/srcbucket21.txt=[part=1/srcbucket21.txt, part=2/srcbucket21.txt], part=2/srcbucket22.txt=[part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=2/srcbucket23.txt=[part=1/srcbucket23.txt, part=2/srcbucket21.txt]} - 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: @@ -286,6 +172,107 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 4 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 2 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 2140 dataSize: 8562 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=1/srcbucket20.txt=[part=1/srcbucket20.txt, part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt, part=1/srcbucket23.txt, part=2/srcbucket21.txt], part=2/srcbucket20.txt=[part=1/srcbucket20.txt, part=2/srcbucket20.txt], part=2/srcbucket21.txt=[part=1/srcbucket21.txt, part=2/srcbucket21.txt], part=2/srcbucket22.txt=[part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=2/srcbucket23.txt=[part=1/srcbucket23.txt, part=2/srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -459,124 +446,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (AND (AND (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) (. (TOK_TABLE_OR_COL b) part))) (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL a) part))) (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL b) part))))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 4 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 2 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 2140 dataSize: 8562 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key], Column[part]] - 1 [Column[key], Column[part]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {part=1/srcbucket20.txt=[part=1/srcbucket20.txt, part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt, part=1/srcbucket23.txt, part=2/srcbucket21.txt], part=2/srcbucket20.txt=[part=1/srcbucket20.txt, part=2/srcbucket20.txt], part=2/srcbucket21.txt=[part=1/srcbucket21.txt, part=2/srcbucket21.txt], part=2/srcbucket22.txt=[part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=2/srcbucket23.txt=[part=1/srcbucket23.txt, part=2/srcbucket21.txt]} - 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: @@ -621,6 +494,107 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 4 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 2 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 2140 dataSize: 8562 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=1/srcbucket20.txt=[part=1/srcbucket20.txt, part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt, part=1/srcbucket23.txt, part=2/srcbucket21.txt], part=2/srcbucket20.txt=[part=1/srcbucket20.txt, part=2/srcbucket20.txt], part=2/srcbucket21.txt=[part=1/srcbucket21.txt, part=2/srcbucket21.txt], part=2/srcbucket22.txt=[part=1/srcbucket22.txt, part=2/srcbucket20.txt], part=2/srcbucket23.txt=[part=1/srcbucket23.txt, part=2/srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin12.q.out ql/src/test/results/clientpositive/bucketmapjoin12.q.out index aa50cfe..2c48acd 100644 --- ql/src/test/results/clientpositive/bucketmapjoin12.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin12.q.out @@ -92,82 +92,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) '1')) (= (. (TOK_TABLE_OR_COL b) part) '1')))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - 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 {part=1/srcbucket20.txt=[part=1/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -212,6 +140,65 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=1/srcbucket20.txt=[part=1/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -341,75 +328,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_3) b) (and (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) '1')) (= (. (TOK_TABLE_OR_COL b) part) '1')))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_3 - numFiles 2 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_3 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_3 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_3 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_3 - name: default.srcbucket_mapjoin_part_3 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - 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: @@ -453,6 +375,58 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_3 + numFiles 2 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_3 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_3 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_3 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_3 + name: default.srcbucket_mapjoin_part_3 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin13.q.out ql/src/test/results/clientpositive/bucketmapjoin13.q.out index eceeade..5be19d0 100644 --- ql/src/test/results/clientpositive/bucketmapjoin13.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin13.q.out @@ -98,76 +98,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (= (. (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 500 - partition_columns part - rawDataSize 5312 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE - 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: @@ -211,6 +145,59 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 500 + partition_columns part + rawDataSize 5312 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -398,83 +385,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) '2')))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 500 - partition_columns part - rawDataSize 5312 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE - 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 {part=2/000000_0=[part=1/000000_0], part=2/000001_0=[part=1/000001_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: @@ -519,6 +433,66 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 500 + partition_columns part + rawDataSize 5312 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=2/000000_0=[part=1/000000_0], part=2/000001_0=[part=1/000001_0]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -674,83 +648,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (= (. (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 500 - partition_columns part - rawDataSize 5312 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE - 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 {part=2/000000_0=[part=1/000000_0], part=2/000001_0=[part=1/000001_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: @@ -795,6 +696,66 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 500 + partition_columns part + rawDataSize 5312 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=2/000000_0=[part=1/000000_0], part=2/000001_0=[part=1/000001_0]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -952,83 +913,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (= (. (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 500 - partition_columns part - rawDataSize 5312 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE - 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 {part=2/000000_0=[part=1/000000_0], part=2/000001_0=[part=1/000001_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: @@ -1073,6 +961,66 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 500 + partition_columns part + rawDataSize 5312 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=2/000000_0=[part=1/000000_0], part=2/000001_0=[part=1/000001_0]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin2.q.out ql/src/test/results/clientpositive/bucketmapjoin2.q.out index 1dfe3e7..61bc7e7 100644 --- ql/src/test/results/clientpositive/bucketmapjoin2.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin2.q.out @@ -77,8 +77,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-8 depends on stages: Stage-1 , consists of Stage-5, Stage-4, Stage-6 Stage-5 Stage-0 depends on stages: Stage-5, Stage-4, Stage-7 @@ -88,78 +87,6 @@ STAGE DEPENDENCIES: Stage-7 depends on stages: Stage-6 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 3062 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 29 dataSize: 3062 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {ds=2008-04-08/srcbucket20.txt=[ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket21.txt=[ds=2008-04-08/srcbucket23.txt], ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket23.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -223,6 +150,66 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 3062 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 29 dataSize: 3062 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds=2008-04-08/srcbucket20.txt=[ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket21.txt=[ds=2008-04-08/srcbucket23.txt], ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket23.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -621,8 +608,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -632,78 +618,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part { 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: - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part - name: default.srcbucket_mapjoin_part - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 55 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket20.txt, ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket21.txt, ds=2008-04-08/srcbucket23.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -772,6 +686,66 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part { 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: + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part + name: default.srcbucket_mapjoin_part + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 55 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket20.txt, ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket21.txt, ds=2008-04-08/srcbucket23.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1364,8 +1338,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-8 depends on stages: Stage-1 , consists of Stage-5, Stage-4, Stage-6 Stage-5 Stage-0 depends on stages: Stage-5, Stage-4, Stage-7 @@ -1375,119 +1348,6 @@ STAGE DEPENDENCIES: Stage-7 depends on stages: Stage-6 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 3062 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 3062 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 58 dataSize: 6124 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {ds=2008-04-08/srcbucket20.txt=[ds=2008-04-08/srcbucket22.txt, ds=2008-04-09/srcbucket22.txt], ds=2008-04-08/srcbucket21.txt=[ds=2008-04-08/srcbucket23.txt, ds=2008-04-09/srcbucket23.txt], ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket22.txt, ds=2008-04-09/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket23.txt, ds=2008-04-09/srcbucket23.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -1556,6 +1416,107 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 3062 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 3062 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 58 dataSize: 6124 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds=2008-04-08/srcbucket20.txt=[ds=2008-04-08/srcbucket22.txt, ds=2008-04-09/srcbucket22.txt], ds=2008-04-08/srcbucket21.txt=[ds=2008-04-08/srcbucket23.txt, ds=2008-04-09/srcbucket23.txt], ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket22.txt, ds=2008-04-09/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket23.txt, ds=2008-04-09/srcbucket23.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin3.q.out ql/src/test/results/clientpositive/bucketmapjoin3.q.out index ba1962d..630a545 100644 --- ql/src/test/results/clientpositive/bucketmapjoin3.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin3.q.out @@ -94,8 +94,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (and (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")) (= (. (TOK_TABLE_OR_COL a) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-8 depends on stages: Stage-1 , consists of Stage-5, Stage-4, Stage-6 Stage-5 Stage-0 depends on stages: Stage-5, Stage-4, Stage-7 @@ -105,78 +104,6 @@ STAGE DEPENDENCIES: Stage-7 depends on stages: Stage-6 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part { 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: - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part - name: default.srcbucket_mapjoin_part - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 55 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket20.txt, ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket21.txt, ds=2008-04-08/srcbucket23.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -240,6 +167,66 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part { 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: + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part + name: default.srcbucket_mapjoin_part + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 55 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket20.txt, ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket21.txt, ds=2008-04-08/srcbucket23.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -638,8 +625,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (and (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")) (= (. (TOK_TABLE_OR_COL a) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -649,78 +635,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 3062 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 29 dataSize: 3062 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcbucket20.txt=[ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket21.txt=[ds=2008-04-08/srcbucket23.txt], ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket23.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -789,6 +703,66 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 3062 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 29 dataSize: 3062 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcbucket20.txt=[ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket21.txt=[ds=2008-04-08/srcbucket23.txt], ds=2008-04-08/srcbucket22.txt=[ds=2008-04-08/srcbucket22.txt], ds=2008-04-08/srcbucket23.txt=[ds=2008-04-08/srcbucket23.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin4.q.out ql/src/test/results/clientpositive/bucketmapjoin4.q.out index a9c4bce..76b24b2 100644 --- ql/src/test/results/clientpositive/bucketmapjoin4.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin4.q.out @@ -94,8 +94,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-8 depends on stages: Stage-1 , consists of Stage-5, Stage-4, Stage-6 Stage-5 Stage-0 depends on stages: Stage-5, Stage-4, Stage-7 @@ -105,36 +104,6 @@ STAGE DEPENDENCIES: Stage-7 depends on stages: Stage-6 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -198,6 +167,24 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -585,8 +572,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -596,36 +582,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -694,6 +650,24 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin5.q.out ql/src/test/results/clientpositive/bucketmapjoin5.q.out index 4812a21..da826f5 100644 --- ql/src/test/results/clientpositive/bucketmapjoin5.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin5.q.out @@ -132,8 +132,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -143,36 +142,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcbucket20.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket21.txt=[srcbucket21.txt], ds=2008-04-08/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket23.txt=[srcbucket21.txt], ds=2008-04-09/srcbucket20.txt=[srcbucket20.txt], ds=2008-04-09/srcbucket21.txt=[srcbucket21.txt], ds=2008-04-09/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-09/srcbucket23.txt=[srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -236,6 +205,24 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcbucket20.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket21.txt=[srcbucket21.txt], ds=2008-04-08/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket23.txt=[srcbucket21.txt], ds=2008-04-09/srcbucket20.txt=[srcbucket20.txt], ds=2008-04-09/srcbucket21.txt=[srcbucket21.txt], ds=2008-04-09/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-09/srcbucket23.txt=[srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -678,8 +665,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -689,36 +675,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket23.txt=[srcbucket21.txt], ds=2008-04-09/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-09/srcbucket23.txt=[srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -787,6 +743,24 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket23.txt=[srcbucket21.txt], ds=2008-04-09/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-09/srcbucket23.txt=[srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin7.q.out ql/src/test/results/clientpositive/bucketmapjoin7.q.out index f8ea2e2..d978a4b 100644 --- ql/src/test/results/clientpositive/bucketmapjoin7.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin7.q.out @@ -56,84 +56,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (AND (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL b) ds) '2008-04-08')))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_LIMIT 1))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - hr 0 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns ds/hr - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns ds/hr - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {ds=2008-04-08/hr=0/srcbucket20.txt=[ds=2008-04-08/hr=0/srcbucket20.txt], ds=2008-04-08/hr=0/srcbucket21.txt=[ds=2008-04-08/hr=0/srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -194,6 +120,67 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 0 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns ds/hr + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns ds/hr + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {ds=2008-04-08/hr=0/srcbucket20.txt=[ds=2008-04-08/hr=0/srcbucket20.txt], ds=2008-04-08/hr=0/srcbucket21.txt=[ds=2008-04-08/hr=0/srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin8.q.out ql/src/test/results/clientpositive/bucketmapjoin8.q.out index 8e2be73..0e8f7d4 100644 --- ql/src/test/results/clientpositive/bucketmapjoin8.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin8.q.out @@ -64,83 +64,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) '1')) (= (. (TOK_TABLE_OR_COL b) part) '1')))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 3 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - 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 {part=1/srcbucket20.txt=[part=1/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -185,6 +112,66 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 3 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=1/srcbucket20.txt=[part=1/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -322,83 +309,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) '1')) (= (. (TOK_TABLE_OR_COL b) part) '1')))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name value - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - 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 {part=1/srcbucket20.txt=[part=1/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -443,6 +357,66 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name value + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=1/srcbucket20.txt=[part=1/srcbucket20.txt], part=1/srcbucket21.txt=[part=1/srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin9.q.out ql/src/test/results/clientpositive/bucketmapjoin9.q.out index 38b9c87..9ce7ac6 100644 --- ql/src/test/results/clientpositive/bucketmapjoin9.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin9.q.out @@ -70,76 +70,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) '1')) (= (. (TOK_TABLE_OR_COL b) part) '1')))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 3 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 3 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 4200 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 1050 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - 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: @@ -183,6 +117,59 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 3 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 3 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 4200 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 1050 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -349,76 +336,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (AND (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) '1')) (= (. (TOK_TABLE_OR_COL b) part) '1')))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name value - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns part - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 2750 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - 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: @@ -462,6 +383,59 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name value + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns part + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 687 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out index 0139546..bc4605b 100644 --- ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out @@ -60,8 +60,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-8 depends on stages: Stage-1 , consists of Stage-5, Stage-4, Stage-6 Stage-5 Stage-0 depends on stages: Stage-5, Stage-4, Stage-7 @@ -71,71 +70,6 @@ STAGE DEPENDENCIES: Stage-7 depends on stages: Stage-6 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 3 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - numFiles 3 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 4200 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 3 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part - name: default.srcbucket_mapjoin_part - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 40 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -198,6 +132,59 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 3 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + numFiles 3 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 4200 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 3 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part + name: default.srcbucket_mapjoin_part + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 40 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out index bf00114..2518c6f 100644 --- ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out @@ -67,8 +67,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-8 depends on stages: Stage-1 , consists of Stage-5, Stage-4, Stage-6 Stage-5 Stage-0 depends on stages: Stage-5, Stage-4, Stage-7 @@ -78,119 +77,6 @@ STAGE DEPENDENCIES: Stage-7 depends on stages: Stage-6 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 3062 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 3062 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 58 dataSize: 6124 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {srcbucket20.txt=[ds=2008-04-08/srcbucket22.txt, ds=2008-04-09/srcbucket22.txt], srcbucket21.txt=[ds=2008-04-08/srcbucket23.txt, ds=2008-04-09/srcbucket23.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -254,6 +140,107 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 3062 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 3062 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 58 dataSize: 6124 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {srcbucket20.txt=[ds=2008-04-08/srcbucket22.txt, ds=2008-04-09/srcbucket22.txt], srcbucket21.txt=[ds=2008-04-08/srcbucket23.txt, ds=2008-04-09/srcbucket23.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out index 2cc1a0f..9229d37 100644 --- ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out +++ ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out @@ -116,41 +116,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test1) L) (TOK_TABREF (TOK_TABNAME test1) R) (AND (= (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL R) key)) (= (. (TOK_TABLE_OR_COL L) value) (. (TOK_TABLE_OR_COL R) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - r {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt], srcbucket22.txt=[srcbucket22.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -212,6 +181,24 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + r {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt], srcbucket22.txt=[srcbucket22.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -276,41 +263,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test2) L) (TOK_TABREF (TOK_TABNAME test2) R) (AND (= (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL R) key)) (= (. (TOK_TABLE_OR_COL L) value) (. (TOK_TABLE_OR_COL R) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - r {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt], srcbucket22.txt=[srcbucket22.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -372,6 +328,24 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + r {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt], srcbucket22.txt=[srcbucket22.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -438,34 +412,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test1) L) (TOK_TABREF (TOK_TABNAME test1) R) (= (+ (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL L) key)) (. (TOK_TABLE_OR_COL R) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [GenericUDFOPPlus(Column[key], Column[key])] - 1 [GenericUDFBridge(Column[key])] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -526,6 +476,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -590,34 +551,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test1) L) (TOK_TABREF (TOK_TABNAME test2) R) (AND (= (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL R) key)) (= (. (TOK_TABLE_OR_COL L) value) (. (TOK_TABLE_OR_COL R) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -678,6 +615,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -742,34 +690,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test1) L) (TOK_TABREF (TOK_TABNAME test3) R) (AND (= (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL R) key)) (= (. (TOK_TABLE_OR_COL L) value) (. (TOK_TABLE_OR_COL R) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -830,6 +754,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -894,34 +829,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test1) L) (TOK_TABREF (TOK_TABNAME test4) R) (AND (= (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL R) key)) (= (. (TOK_TABLE_OR_COL L) value) (. (TOK_TABLE_OR_COL R) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -982,6 +893,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1046,34 +968,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test2) L) (TOK_TABREF (TOK_TABNAME test3) R) (AND (= (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL R) key)) (= (. (TOK_TABLE_OR_COL L) value) (. (TOK_TABLE_OR_COL R) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -1134,6 +1032,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1198,34 +1107,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test2) L) (TOK_TABREF (TOK_TABNAME test4) R) (AND (= (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL R) key)) (= (. (TOK_TABLE_OR_COL L) value) (. (TOK_TABLE_OR_COL R) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -1286,6 +1171,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1350,34 +1246,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test3) L) (TOK_TABREF (TOK_TABNAME test4) R) (AND (= (. (TOK_TABLE_OR_COL L) key) (. (TOK_TABLE_OR_COL R) key)) (= (. (TOK_TABLE_OR_COL L) value) (. (TOK_TABLE_OR_COL R) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST R))) (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - r - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - r - TableScan - alias: r - Statistics: - numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -1438,6 +1310,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + r + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + r + TableScan + alias: r + Statistics: + numRows: 21 dataSize: 4200 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/bucketsortoptimize_insert_2.q.out ql/src/test/results/clientpositive/bucketsortoptimize_insert_2.q.out index 0014891..c69811f 100644 --- ql/src/test/results/clientpositive/bucketsortoptimize_insert_2.q.out +++ ql/src/test/results/clientpositive/bucketsortoptimize_insert_2.q.out @@ -285,39 +285,17 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test_table1) a) (TOK_TABREF (TOK_TABNAME test_table2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME test_table3) (TOK_PARTSPEC (TOK_PARTVAL ds '1')))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL b) value)))) (TOK_WHERE (and (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL a) ds)) (= (. (TOK_TABLE_OR_COL b) ds) '1'))))) STAGE DEPENDENCIES: - Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-6 is a root stage , consists of Stage-4, Stage-5, Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-3 depends on stages: Stage-0 - Stage-8 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-8 + Stage-5 has a backup stage: Stage-1 Stage-1 STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -359,6 +337,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Reduce Operator Tree: Extract File Output Operator @@ -385,26 +371,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -446,6 +412,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/bucketsortoptimize_insert_4.q.out ql/src/test/results/clientpositive/bucketsortoptimize_insert_4.q.out index 14d0b6a..745fbf2 100644 --- ql/src/test/results/clientpositive/bucketsortoptimize_insert_4.q.out +++ ql/src/test/results/clientpositive/bucketsortoptimize_insert_4.q.out @@ -266,39 +266,17 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test_table1) a) (TOK_TABREF (TOK_TABNAME test_table2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME test_table3) (TOK_PARTSPEC (TOK_PARTVAL ds '1')))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value))) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL a) ds) '1') (= (. (TOK_TABLE_OR_COL b) ds) '1'))))) STAGE DEPENDENCIES: - Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-6 is a root stage , consists of Stage-4, Stage-5, Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-3 depends on stages: Stage-0 - Stage-8 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-8 + Stage-5 has a backup stage: Stage-1 Stage-1 STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -340,6 +318,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Reduce Operator Tree: Extract File Output Operator @@ -366,26 +352,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -427,6 +393,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/bucketsortoptimize_insert_5.q.out ql/src/test/results/clientpositive/bucketsortoptimize_insert_5.q.out index e83ef59..404ab41 100644 --- ql/src/test/results/clientpositive/bucketsortoptimize_insert_5.q.out +++ ql/src/test/results/clientpositive/bucketsortoptimize_insert_5.q.out @@ -71,39 +71,17 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test_table1) a) (TOK_TABREF (TOK_TABNAME test_table2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME test_table3) (TOK_PARTSPEC (TOK_PARTVAL ds '1')))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL b) value)))) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL a) ds) '1') (= (. (TOK_TABLE_OR_COL b) ds) '1'))))) STAGE DEPENDENCIES: - Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-6 is a root stage , consists of Stage-4, Stage-5, Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-3 depends on stages: Stage-0 - Stage-8 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-8 + Stage-5 has a backup stage: Stage-1 Stage-1 STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -145,6 +123,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Reduce Operator Tree: Extract File Output Operator @@ -171,26 +157,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -232,6 +198,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/bucketsortoptimize_insert_6.q.out ql/src/test/results/clientpositive/bucketsortoptimize_insert_6.q.out index b5828f0..f3a86ed 100644 --- ql/src/test/results/clientpositive/bucketsortoptimize_insert_6.q.out +++ ql/src/test/results/clientpositive/bucketsortoptimize_insert_6.q.out @@ -448,39 +448,17 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test_table1) a) (TOK_TABREF (TOK_TABNAME test_table2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) key2) (. (TOK_TABLE_OR_COL b) key2))))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME test_table3) (TOK_PARTSPEC (TOK_PARTVAL ds '1')))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL b) value)))) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL a) ds) '1') (= (. (TOK_TABLE_OR_COL b) ds) '1'))))) STAGE DEPENDENCIES: - Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-6 is a root stage , consists of Stage-4, Stage-5, Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-3 depends on stages: Stage-0 - Stage-8 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-8 + Stage-5 has a backup stage: Stage-1 Stage-1 STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} {key2} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[key2]] - 1 [Column[key], Column[key2]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -530,6 +508,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Reduce Operator Tree: Extract File Output Operator @@ -556,26 +542,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {key2} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[key2]] - 1 [Column[key], Column[key2]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -625,6 +591,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Reduce Operator Tree: Extract File Output Operator @@ -732,39 +706,17 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test_table1) a) (TOK_TABREF (TOK_TABNAME test_table2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) key2) (. (TOK_TABLE_OR_COL b) key2))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key2)) (TOK_SELEXPR (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL b) value)) value)) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL a) ds) '1') (= (. (TOK_TABLE_OR_COL b) ds) '1'))))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME test_table3) (TOK_PARTSPEC (TOK_PARTVAL ds '1')))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) value))))) STAGE DEPENDENCIES: - Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-6 is a root stage , consists of Stage-4, Stage-5, Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 - Stage-8 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-8 + Stage-5 has a backup stage: Stage-1 Stage-1 STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - subq1:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq1:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} {key2} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[key2]] - 1 [Column[key], Column[key2]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -814,6 +766,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq1:b + TableScan + alias: b Reduce Operator Tree: Extract File Output Operator @@ -840,26 +800,6 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - subq1:a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq1:a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {key2} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[key2]] - 1 [Column[key], Column[key2]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -909,6 +849,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq1:a + TableScan + alias: a Reduce Operator Tree: Extract File Output Operator @@ -1503,39 +1451,17 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test_table1) a) (TOK_TABREF (TOK_TABNAME test_table2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) key2) (. (TOK_TABLE_OR_COL b) key2))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key2)) (TOK_SELEXPR (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL b) value)) value)) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL a) ds) '1') (= (. (TOK_TABLE_OR_COL b) ds) '1'))))) subq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) key2) k1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) key) k2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) value))))) subq2)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME test_table4) (TOK_PARTSPEC (TOK_PARTVAL ds '1')))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq2) k2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq2) k1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq2) value))))) STAGE DEPENDENCIES: - Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 + Stage-6 is a root stage , consists of Stage-4, Stage-5, Stage-1 + Stage-4 has a backup stage: Stage-1 Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-3 depends on stages: Stage-0 - Stage-8 has a backup stage: Stage-1 - Stage-5 depends on stages: Stage-8 + Stage-5 has a backup stage: Stage-1 Stage-1 STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - subq2:subq1:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq2:subq1:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} {key2} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[key2]] - 1 [Column[key], Column[key2]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -1585,6 +1511,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq2:subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq2:subq1:b + TableScan + alias: b Reduce Operator Tree: Extract File Output Operator @@ -1611,26 +1545,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - subq2:subq1:a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq2:subq1:a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {key2} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[key2]] - 1 [Column[key], Column[key2]] - Position of Big Table: 1 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -1680,6 +1594,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq2:subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq2:subq1:a + TableScan + alias: a Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/correlationoptimizer1.q.out ql/src/test/results/clientpositive/correlationoptimizer1.q.out index 3ef5cf2..e033745 100644 --- ql/src/test/results/clientpositive/correlationoptimizer1.q.out +++ ql/src/test/results/clientpositive/correlationoptimizer1.q.out @@ -376,32 +376,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-2 depends on stages: Stage-6 + Stage-2 is a root stage Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp:x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -448,6 +427,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:x + TableScan + alias: x Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/correlationoptimizer3.q.out ql/src/test/results/clientpositive/correlationoptimizer3.q.out index e633287..1f53509 100644 --- ql/src/test/results/clientpositive/correlationoptimizer3.q.out +++ ql/src/test/results/clientpositive/correlationoptimizer3.q.out @@ -608,47 +608,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) value) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value))))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-2 depends on stages: Stage-9 + Stage-2 is a root stage Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp:b:x - Fetch Operator - limit: -1 - tmp:d:x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp:b:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - tmp:d:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -729,6 +693,20 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp:b:x + Fetch Operator + limit: -1 + tmp:d:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:b:x + TableScan + alias: x + tmp:d:x + TableScan + alias: x Reduce Operator Tree: Demux Operator Group By Operator @@ -1474,47 +1452,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) cnt) cnt) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value))))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-2 depends on stages: Stage-9 + Stage-2 is a root stage Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp:b:x - Fetch Operator - limit: -1 - tmp:d:x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp:b:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - tmp:d:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -1595,6 +1537,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp:b:x + Fetch Operator + limit: -1 + tmp:d:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:b:x + TableScan + alias: x + tmp:d:x + TableScan + alias: x Reduce Operator Tree: Demux Operator Mux Operator diff --git ql/src/test/results/clientpositive/correlationoptimizer4.q.out ql/src/test/results/clientpositive/correlationoptimizer4.q.out index c037d3c..90656a6 100644 --- ql/src/test/results/clientpositive/correlationoptimizer4.q.out +++ ql/src/test/results/clientpositive/correlationoptimizer4.q.out @@ -433,51 +433,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-2 depends on stages: Stage-7 + Stage-2 is a root stage Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp:y - Fetch Operator - limit: -1 - tmp:z - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 - 1 {key} - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 0 - tmp:z - TableScan - alias: z - HashTable Sink Operator - condition expressions: - 0 - 1 {key} - 2 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -527,6 +487,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp:y + Fetch Operator + limit: -1 + tmp:z + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:y + TableScan + alias: y + tmp:z + TableScan + alias: z Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/correlationoptimizer5.q.out ql/src/test/results/clientpositive/correlationoptimizer5.q.out index 02ba37a..f1348eb 100644 --- ql/src/test/results/clientpositive/correlationoptimizer5.q.out +++ ql/src/test/results/clientpositive/correlationoptimizer5.q.out @@ -535,40 +535,16 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) x) (TOK_TABREF (TOK_TABNAME T2) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) val))))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T3) m) (TOK_TABREF (TOK_TABNAME T4) n) (= (. (TOK_TABLE_OR_COL m) key) (. (TOK_TABLE_OR_COL n) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL m) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL n) val))))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_co3))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) val))))) STAGE DEPENDENCIES: - Stage-14 is a root stage - Stage-10 depends on stages: Stage-14 - Stage-9 depends on stages: Stage-10, Stage-11 , consists of Stage-12, Stage-13, Stage-2 - Stage-12 has a backup stage: Stage-2 - Stage-7 depends on stages: Stage-12 + Stage-10 is a root stage + Stage-9 depends on stages: Stage-10, Stage-11 , consists of Stage-7, Stage-8, Stage-2 + Stage-7 has a backup stage: Stage-2 Stage-0 depends on stages: Stage-2, Stage-7, Stage-8 Stage-3 depends on stages: Stage-0 - Stage-13 has a backup stage: Stage-2 - Stage-8 depends on stages: Stage-13 + Stage-8 has a backup stage: Stage-2 Stage-2 - Stage-15 is a root stage - Stage-11 depends on stages: Stage-15 + Stage-11 is a root stage STAGE PLANS: - Stage: Stage-14 - Map Reduce Local Work - Alias -> Map Local Tables: - d:m - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - d:m - TableScan - alias: m - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {val} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-10 Map Reduce Alias -> Map Operator Tree: @@ -603,29 +579,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + d:m + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + d:m + TableScan + alias: m Stage: Stage-9 Conditional Operator - Stage: Stage-12 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -660,6 +625,13 @@ STAGE PLANS: name: default.dest_co3 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-0 Move Operator @@ -674,25 +646,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-13 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-8 Map Reduce Alias -> Map Operator Tree: @@ -727,6 +680,13 @@ STAGE PLANS: name: default.dest_co3 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-2 Map Reduce @@ -784,26 +744,6 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.dest_co3 - Stage: Stage-15 - Map Reduce Local Work - Alias -> Map Local Tables: - b:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-11 Map Reduce Alias -> Map Operator Tree: @@ -836,6 +776,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b:y + TableScan + alias: y PREHOOK: query: INSERT OVERWRITE TABLE dest_co3 SELECT b.key, d.val diff --git ql/src/test/results/clientpositive/correlationoptimizer6.q.out ql/src/test/results/clientpositive/correlationoptimizer6.q.out index 557ee9e..907604b 100644 --- ql/src/test/results/clientpositive/correlationoptimizer6.q.out +++ ql/src/test/results/clientpositive/correlationoptimizer6.q.out @@ -673,47 +673,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-2 depends on stages: Stage-9 + Stage-2 is a root stage Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - xx:y - Fetch Operator - limit: -1 - yy:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - xx:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - yy:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -801,6 +765,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + yy:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y + yy:y + TableScan + alias: y Reduce Operator Tree: Demux Operator Group By Operator @@ -5180,34 +5158,12 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL x) value)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) STAGE DEPENDENCIES: - Stage-11 is a root stage - Stage-2 depends on stages: Stage-11 - Stage-10 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-10 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 Stage-4 depends on stages: Stage-3 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-11 - Map Reduce Local Work - Alias -> Map Local Tables: - yy:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - yy:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -5262,6 +5218,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + yy:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + yy:y + TableScan + alias: y Reduce Operator Tree: Group By Operator aggregations: @@ -5291,26 +5255,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - xx:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - xx:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -5375,6 +5319,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y Reduce Operator Tree: Demux Operator Group By Operator diff --git ql/src/test/results/clientpositive/correlationoptimizer7.q.out ql/src/test/results/clientpositive/correlationoptimizer7.q.out index b1fe60a..1f533a5 100644 --- ql/src/test/results/clientpositive/correlationoptimizer7.q.out +++ ql/src/test/results/clientpositive/correlationoptimizer7.q.out @@ -18,33 +18,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src1) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-4 depends on stages: Stage-9 - Stage-8 depends on stages: Stage-4 - Stage-2 depends on stages: Stage-8 + Stage-4 is a root stage + Stage-2 depends on stages: Stage-4 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - xx:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - xx:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -91,6 +69,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y Reduce Operator Tree: Group By Operator aggregations: @@ -116,26 +102,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - yy - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - yy - TableScan - alias: yy - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -187,6 +153,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + yy + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + yy + TableScan + alias: yy Reduce Operator Tree: Extract File Output Operator @@ -256,32 +230,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src1) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-1 depends on stages: Stage-6 + Stage-1 is a root stage Stage-2 depends on stages: Stage-1 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - xx:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - xx:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -345,6 +298,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y Reduce Operator Tree: Demux Operator Group By Operator @@ -525,33 +486,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src1) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-4 depends on stages: Stage-9 - Stage-8 depends on stages: Stage-4 - Stage-2 depends on stages: Stage-8 + Stage-4 is a root stage + Stage-2 depends on stages: Stage-4 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - xx:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - xx:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -598,6 +537,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y Reduce Operator Tree: Group By Operator aggregations: @@ -623,26 +570,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - yy - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - yy - TableScan - alias: yy - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -694,6 +621,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + yy + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + yy + TableScan + alias: yy Reduce Operator Tree: Extract File Output Operator @@ -763,32 +698,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src1) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-1 depends on stages: Stage-6 + Stage-1 is a root stage Stage-2 depends on stages: Stage-1 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - xx:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - xx:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -852,6 +766,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y Reduce Operator Tree: Demux Operator Group By Operator diff --git ql/src/test/results/clientpositive/explain_rearrange.q.out ql/src/test/results/clientpositive/explain_rearrange.q.out index 47b1afb..cac642d 100644 --- ql/src/test/results/clientpositive/explain_rearrange.q.out +++ ql/src/test/results/clientpositive/explain_rearrange.q.out @@ -57,12 +57,10 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage [MAPRED] - Stage-9 depends on stages: Stage-1, Stage-5 , consists of Stage-10, Stage-11, Stage-3 [CONDITIONAL] - Stage-10 has a backup stage: Stage-3 [MAPREDLOCAL] - Stage-7 depends on stages: Stage-10 [MAPRED] + Stage-9 depends on stages: Stage-1, Stage-5 , consists of Stage-7, Stage-8, Stage-3 [CONDITIONAL] + Stage-7 has a backup stage: Stage-3 [MAPRED] Stage-4 depends on stages: Stage-3, Stage-7, Stage-8 [MAPRED] - Stage-11 has a backup stage: Stage-3 [MAPREDLOCAL] - Stage-8 depends on stages: Stage-11 [MAPRED] + Stage-8 has a backup stage: Stage-3 [MAPRED] Stage-3 [MAPRED] Stage-5 is a root stage [MAPRED] Stage-0 is a root stage [FETCH] @@ -140,25 +138,6 @@ STAGE PLANS: Stage: Stage-9 Conditional Operator - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -194,6 +173,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-4 Map Reduce @@ -227,25 +213,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-11 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-8 Map Reduce Alias -> Map Operator Tree: @@ -281,6 +248,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-3 Map Reduce @@ -459,15 +433,13 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage [MAPRED] - Stage-2 depends on stages: Stage-1, Stage-9 , consists of Stage-3, Stage-6, Stage-8 [CONDITIONAL] - Stage-3 has a backup stage: Stage-8 [MAPREDLOCAL] - Stage-4 depends on stages: Stage-3 [MAPRED] - Stage-5 depends on stages: Stage-8, Stage-4, Stage-7 [MAPRED] - Stage-6 has a backup stage: Stage-8 [MAPREDLOCAL] - Stage-7 depends on stages: Stage-6 [MAPRED] - Stage-8 [MAPRED] - Stage-9 is a root stage [MAPRED] - Stage-10 is a root stage [FETCH] + Stage-2 depends on stages: Stage-1, Stage-7 , consists of Stage-3, Stage-5, Stage-6 [CONDITIONAL] + Stage-3 has a backup stage: Stage-6 [MAPRED] + Stage-4 depends on stages: Stage-6, Stage-3, Stage-5 [MAPRED] + Stage-5 has a backup stage: Stage-6 [MAPRED] + Stage-6 [MAPRED] + Stage-7 is a root stage [MAPRED] + Stage-8 is a root stage [FETCH] STAGE PLANS: Stage: Stage-1 @@ -543,25 +515,6 @@ STAGE PLANS: Conditional Operator Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: $INTNAME @@ -596,8 +549,15 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan - Stage: Stage-5 + Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: #### A masked pattern was here #### @@ -629,26 +589,7 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - - Stage: Stage-7 + Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: $INTNAME1 @@ -683,8 +624,15 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan - Stage: Stage-8 + Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: $INTNAME @@ -743,7 +691,7 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-9 + Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: src2:subq2:a @@ -812,7 +760,7 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-10 + Stage: Stage-8 Fetch Operator limit: -1 @@ -862,14 +810,12 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage [MAPRED] Stage-2 is a root stage [MAPRED] - Stage-3 depends on stages: Stage-1, Stage-2 , consists of Stage-4, Stage-6, Stage-8 [CONDITIONAL] - Stage-4 has a backup stage: Stage-8 [MAPREDLOCAL] - Stage-5 depends on stages: Stage-4 [MAPRED] - Stage-6 has a backup stage: Stage-8 [MAPREDLOCAL] - Stage-7 depends on stages: Stage-6 [MAPRED] - Stage-8 [MAPRED] - Stage-9 depends on stages: Stage-8, Stage-5, Stage-7 [MAPRED] - Stage-10 is a root stage [FETCH] + Stage-3 depends on stages: Stage-1, Stage-2 , consists of Stage-4, Stage-5, Stage-6 [CONDITIONAL] + Stage-4 has a backup stage: Stage-6 [MAPRED] + Stage-5 has a backup stage: Stage-6 [MAPRED] + Stage-6 [MAPRED] + Stage-7 depends on stages: Stage-6, Stage-4, Stage-5 [MAPRED] + Stage-8 is a root stage [FETCH] STAGE PLANS: Stage: Stage-1 @@ -1014,25 +960,6 @@ STAGE PLANS: Conditional Operator Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: $INTNAME @@ -1067,27 +994,15 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - - Stage: Stage-7 + Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: $INTNAME1 @@ -1122,8 +1037,15 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan - Stage: Stage-8 + Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: $INTNAME @@ -1182,7 +1104,7 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-9 + Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: #### A masked pattern was here #### @@ -1214,7 +1136,7 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-10 + Stage: Stage-8 Fetch Operator limit: -1 @@ -1265,13 +1187,11 @@ STAGE DEPENDENCIES: Stage-1 is a root stage [MAPRED] Stage-2 is a root stage [MAPRED] Stage-3 depends on stages: Stage-1, Stage-2 , consists of Stage-4, Stage-5, Stage-6 [CONDITIONAL] - Stage-4 has a backup stage: Stage-6 [MAPREDLOCAL] - Stage-5 has a backup stage: Stage-6 [MAPREDLOCAL] + Stage-4 has a backup stage: Stage-6 [MAPRED] + Stage-5 has a backup stage: Stage-6 [MAPRED] Stage-6 [MAPRED] - Stage-7 depends on stages: Stage-4 [MAPRED] - Stage-8 depends on stages: Stage-5 [MAPRED] - Stage-9 depends on stages: Stage-6, Stage-7, Stage-8 [MAPRED] - Stage-10 is a root stage [FETCH] + Stage-7 depends on stages: Stage-6, Stage-4, Stage-5 [MAPRED] + Stage-8 is a root stage [FETCH] STAGE PLANS: Stage: Stage-1 @@ -1416,15 +1336,13 @@ STAGE PLANS: Conditional Operator Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME TableScan - HashTable Sink Operator + Map Join Operator + condition map: + Inner Join 0 to 1 condition expressions: 0 {_col0} {_col1} 1 {_col1} @@ -1432,18 +1350,42 @@ STAGE PLANS: keys: 0 [Column[_col0]] 1 [Column[_col0]] + outputColumnNames: _col0, _col1, _col3 Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME + Map Reduce + Alias -> Map Operator Tree: + $INTNAME1 TableScan - HashTable Sink Operator + Map Join Operator + condition map: + Inner Join 0 to 1 condition expressions: 0 {_col0} {_col1} 1 {_col1} @@ -1451,7 +1393,33 @@ STAGE PLANS: keys: 0 [Column[_col0]] 1 [Column[_col0]] + outputColumnNames: _col0, _col1, _col3 Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-6 Map Reduce @@ -1515,78 +1483,6 @@ STAGE PLANS: Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: - $INTNAME - TableScan - Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - outputColumnNames: _col0, _col1, _col3 - Position of Big Table: 0 - Select Operator - expressions: - expr: _col0 - type: int - expr: _col1 - type: bigint - expr: _col3 - type: bigint - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Local Work: - Map Reduce Local Work - - Stage: Stage-8 - Map Reduce - Alias -> Map Operator Tree: - $INTNAME1 - TableScan - Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - outputColumnNames: _col0, _col1, _col3 - Position of Big Table: 1 - Select Operator - expressions: - expr: _col0 - type: int - expr: _col1 - type: bigint - expr: _col3 - type: bigint - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Local Work: - Map Reduce Local Work - - Stage: Stage-9 - Map Reduce - Alias -> Map Operator Tree: #### A masked pattern was here #### TableScan Reduce Output Operator @@ -1616,7 +1512,7 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-10 + Stage: Stage-8 Fetch Operator limit: -1 diff --git ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out index effa6c8..8cdbad3 100644 --- ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out +++ ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out @@ -74,17 +74,17 @@ SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table@part=1 -Execution failed with exit status: 3 +Execution failed with exit status: 2 Obtaining error information Task failed! Task ID: - Stage-8 + Stage-5 Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: -- This test tests the scenario when the mapper dies. So, create a conditional task for the mapjoin. -- Tests a join which is not converted to a map join, the output should be bucketed and sorted. diff --git ql/src/test/results/clientpositive/join25.q.out ql/src/test/results/clientpositive/join25.q.out index 630d862..71fdcee 100644 --- ql/src/test/results/clientpositive/join25.q.out +++ ql/src/test/results/clientpositive/join25.q.out @@ -17,8 +17,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST x))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -28,26 +27,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -85,6 +64,14 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x Stage: Stage-7 Conditional Operator diff --git ql/src/test/results/clientpositive/join26.q.out ql/src/test/results/clientpositive/join26.q.out index 9fca2c3..00677d7 100644 --- ql/src/test/results/clientpositive/join26.q.out +++ ql/src/test/results/clientpositive/join26.q.out @@ -19,8 +19,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL z) key)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST x y))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))))) STAGE DEPENDENCIES: - Stage-10 is a root stage - Stage-1 depends on stages: Stage-10 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -30,51 +29,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - Statistics: - numRows: 2 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - y - TableScan - alias: y - Statistics: - numRows: 29 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -140,6 +94,26 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + Statistics: + numRows: 2 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + y + TableScan + alias: y + Statistics: + numRows: 29 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/join27.q.out ql/src/test/results/clientpositive/join27.q.out index 0ad8bff..0ae3160 100644 --- ql/src/test/results/clientpositive/join27.q.out +++ ql/src/test/results/clientpositive/join27.q.out @@ -17,8 +17,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) value) (. (TOK_TABLE_OR_COL y) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST x))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -28,26 +27,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[value]] - 1 [Column[value]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -85,6 +64,14 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x Stage: Stage-7 Conditional Operator diff --git ql/src/test/results/clientpositive/join28.q.out ql/src/test/results/clientpositive/join28.q.out index f50072c..f61ab5e 100644 --- ql/src/test/results/clientpositive/join28.q.out +++ ql/src/test/results/clientpositive/join28.q.out @@ -27,47 +27,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value2)))) subq) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL subq) key1) (. (TOK_TABLE_OR_COL z) key)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value))))) STAGE DEPENDENCIES: - Stage-8 is a root stage - Stage-6 depends on stages: Stage-8 + Stage-6 is a root stage Stage-0 depends on stages: Stage-6 Stage-3 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - subq:x - Fetch Operator - limit: -1 - z - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - z - TableScan - alias: z - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -120,6 +84,20 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq:x + Fetch Operator + limit: -1 + z + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq:x + TableScan + alias: x + z + TableScan + alias: z Stage: Stage-0 Move Operator diff --git ql/src/test/results/clientpositive/join29.q.out ql/src/test/results/clientpositive/join29.q.out index 0e0bbb4..802b67c 100644 --- ql/src/test/results/clientpositive/join29.q.out +++ ql/src/test/results/clientpositive/join29.q.out @@ -24,13 +24,11 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-7 depends on stages: Stage-1, Stage-4 , consists of Stage-8, Stage-9, Stage-2 - Stage-8 has a backup stage: Stage-2 - Stage-5 depends on stages: Stage-8 + Stage-7 depends on stages: Stage-1, Stage-4 , consists of Stage-5, Stage-6, Stage-2 + Stage-5 has a backup stage: Stage-2 Stage-0 depends on stages: Stage-2, Stage-5, Stage-6 Stage-3 depends on stages: Stage-0 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 + Stage-6 has a backup stage: Stage-2 Stage-2 Stage-4 is a root stage @@ -95,25 +93,6 @@ STAGE PLANS: Stage: Stage-7 Conditional Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -150,6 +129,13 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-0 Move Operator @@ -164,25 +150,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -219,6 +186,13 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-2 Map Reduce diff --git ql/src/test/results/clientpositive/join30.q.out ql/src/test/results/clientpositive/join30.q.out index 924579c..1510d72 100644 --- ql/src/test/results/clientpositive/join30.q.out +++ ql/src/test/results/clientpositive/join30.q.out @@ -15,32 +15,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST x))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) STAGE DEPENDENCIES: - Stage-4 is a root stage - Stage-1 depends on stages: Stage-4 + Stage-1 is a root stage Stage-0 depends on stages: Stage-1 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -87,6 +66,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/join31.q.out ql/src/test/results/clientpositive/join31.q.out index 85b0196..0cd6380 100644 --- ql/src/test/results/clientpositive/join31.q.out +++ ql/src/test/results/clientpositive/join31.q.out @@ -26,14 +26,12 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-8 depends on stages: Stage-1, Stage-5 , consists of Stage-9, Stage-10, Stage-2 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 + Stage-8 depends on stages: Stage-1, Stage-5 , consists of Stage-6, Stage-7, Stage-2 + Stage-6 has a backup stage: Stage-2 Stage-3 depends on stages: Stage-2, Stage-6, Stage-7 Stage-0 depends on stages: Stage-3 Stage-4 depends on stages: Stage-0 - Stage-10 has a backup stage: Stage-2 - Stage-7 depends on stages: Stage-10 + Stage-7 has a backup stage: Stage-2 Stage-2 Stage-5 is a root stage @@ -96,25 +94,6 @@ STAGE PLANS: Stage: Stage-8 Conditional Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -155,6 +134,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-3 Map Reduce @@ -212,25 +198,6 @@ STAGE PLANS: Stage: Stage-4 Stats-Aggr Operator - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -271,6 +238,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-2 Map Reduce diff --git ql/src/test/results/clientpositive/join32.q.out ql/src/test/results/clientpositive/join32.q.out index 9e84d84..729407d 100644 --- ql/src/test/results/clientpositive/join32.q.out +++ ql/src/test/results/clientpositive/join32.q.out @@ -23,95 +23,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL x) value) (. (TOK_TABLE_OR_COL z) value)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-5 depends on stages: Stage-7 + Stage-5 is a root stage Stage-0 depends on stages: Stage-5 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - z - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: hr=11 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - hr 11 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 0 - partition_columns ds/hr - rawDataSize 0 - serialization.ddl struct srcpart { string 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: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - Statistics: - numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - z - TableScan - alias: z - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col5} {_col0} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -188,6 +104,68 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + z + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: hr=11 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 0 + partition_columns ds/hr + rawDataSize 0 + serialization.ddl struct srcpart { string 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: + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + Statistics: + numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + z + TableScan + alias: z + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/join32_lessSize.q.out ql/src/test/results/clientpositive/join32_lessSize.q.out index 0f6f8ec..3a16f9c 100644 --- ql/src/test/results/clientpositive/join32_lessSize.q.out +++ ql/src/test/results/clientpositive/join32_lessSize.q.out @@ -28,37 +28,12 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL x) value) (. (TOK_TABLE_OR_COL z) value)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))))) STAGE DEPENDENCIES: - Stage-8 is a root stage - Stage-6 depends on stages: Stage-8 - Stage-7 depends on stages: Stage-6 - Stage-5 depends on stages: Stage-7 + Stage-6 is a root stage + Stage-5 depends on stages: Stage-6 Stage-0 depends on stages: Stage-5 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - Statistics: - numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -101,6 +76,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + Statistics: + numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -191,71 +177,6 @@ STAGE PLANS: Truncated Path -> Alias: /src [y] - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - z - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: hr=11 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - hr 11 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 0 - partition_columns ds/hr - rawDataSize 0 - serialization.ddl struct srcpart { string 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: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart - Alias -> Map Local Operator Tree: - z - TableScan - alias: z - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col5} {_col0} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -315,6 +236,59 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + z + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: hr=11 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 0 + partition_columns ds/hr + rawDataSize 0 + serialization.ddl struct srcpart { string 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: + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart + Alias -> Map Local Operator Tree: + z + TableScan + alias: z + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -548,43 +522,16 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) w) (TOK_TABREF (TOK_TABNAME src1) x) (= (. (TOK_TABLE_OR_COL x) value) (. (TOK_TABLE_OR_COL w) value))) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME src1) z) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))))) STAGE DEPENDENCIES: - Stage-15 is a root stage - Stage-11 depends on stages: Stage-15 - Stage-10 depends on stages: Stage-11 , consists of Stage-12, Stage-13, Stage-14, Stage-2 - Stage-12 has a backup stage: Stage-2 - Stage-7 depends on stages: Stage-12 + Stage-11 is a root stage + Stage-10 depends on stages: Stage-11 , consists of Stage-7, Stage-8, Stage-9, Stage-2 + Stage-7 has a backup stage: Stage-2 Stage-0 depends on stages: Stage-2, Stage-7, Stage-8, Stage-9 Stage-3 depends on stages: Stage-0 - Stage-13 has a backup stage: Stage-2 - Stage-8 depends on stages: Stage-13 - Stage-14 has a backup stage: Stage-2 - Stage-9 depends on stages: Stage-14 + Stage-8 has a backup stage: Stage-2 + Stage-9 has a backup stage: Stage-2 Stage-2 STAGE PLANS: - Stage: Stage-15 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - Statistics: - numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[value]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-11 Map Reduce Alias -> Map Operator Tree: @@ -627,6 +574,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + Statistics: + numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -720,47 +678,6 @@ STAGE PLANS: Stage: Stage-10 Conditional Operator - Stage: Stage-12 - Map Reduce Local Work - Alias -> Map Local Tables: - y - Fetch Operator - limit: -1 - z - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - y - TableScan - alias: y - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col4} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[_col4]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 0 - z - TableScan - alias: z - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col4} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[_col4]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 0 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -822,6 +739,22 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + y + Fetch Operator + limit: -1 + z + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + y + TableScan + alias: y + GatherStats: false + z + TableScan + alias: z + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -962,46 +895,6 @@ STAGE PLANS: Stats-Aggr Operator #### A masked pattern was here #### - Stage: Stage-13 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - z - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col4} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[_col4]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 1 - z - TableScan - alias: z - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col4} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[_col4]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 1 - Stage: Stage-8 Map Reduce Alias -> Map Operator Tree: @@ -1064,6 +957,21 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + z + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + GatherStats: false + z + TableScan + alias: z + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1174,46 +1082,6 @@ STAGE PLANS: Truncated Path -> Alias: /src [y] - Stage: Stage-14 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col4} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[_col4]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - y - TableScan - alias: y - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col4} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[_col4]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - Stage: Stage-9 Map Reduce Alias -> Map Operator Tree: @@ -1276,6 +1144,21 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + GatherStats: false + y + TableScan + alias: y + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1759,37 +1642,12 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) res) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL res) value) (. (TOK_TABLE_OR_COL z) value)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j2))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL res) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL res) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-7 depends on stages: Stage-9 - Stage-8 depends on stages: Stage-7 - Stage-6 depends on stages: Stage-8 + Stage-7 is a root stage + Stage-6 depends on stages: Stage-7 Stage-0 depends on stages: Stage-6 Stage-3 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - res:x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - res:x - TableScan - alias: x - Statistics: - numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -1841,6 +1699,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + res:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + res:x + TableScan + alias: x + Statistics: + numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -1931,71 +1800,6 @@ STAGE PLANS: Truncated Path -> Alias: /src [res:y] - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - z - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: hr=11 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - hr 11 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 0 - partition_columns ds/hr - rawDataSize 0 - serialization.ddl struct srcpart { string 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: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart - Alias -> Map Local Operator Tree: - z - TableScan - alias: z - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -2055,6 +1859,59 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + z + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: hr=11 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 0 + partition_columns ds/hr + rawDataSize 0 + serialization.ddl struct srcpart { string 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: + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart + Alias -> Map Local Operator Tree: + z + TableScan + alias: z + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2304,37 +2161,12 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) res) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL res) value) (. (TOK_TABLE_OR_COL z) value)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j2))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL res) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL res) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-7 depends on stages: Stage-9 - Stage-8 depends on stages: Stage-7 - Stage-6 depends on stages: Stage-8 + Stage-7 is a root stage + Stage-6 depends on stages: Stage-7 Stage-0 depends on stages: Stage-6 Stage-3 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - res:y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - res:y - TableScan - alias: y - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -2386,6 +2218,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + res:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + res:y + TableScan + alias: y + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2476,71 +2319,6 @@ STAGE PLANS: Truncated Path -> Alias: /src1 [res:x] - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - z - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: hr=11 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - hr 11 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 0 - partition_columns ds/hr - rawDataSize 0 - serialization.ddl struct srcpart { string 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: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart - Alias -> Map Local Operator Tree: - z - TableScan - alias: z - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -2605,6 +2383,59 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + z + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: hr=11 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 0 + partition_columns ds/hr + rawDataSize 0 + serialization.ddl struct srcpart { string 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: + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart + Alias -> Map Local Operator Tree: + z + TableScan + alias: z + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -2880,34 +2711,12 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) res) (TOK_TABREF (TOK_TABNAME srcpart) x) (and (and (= (. (TOK_TABLE_OR_COL res) value) (. (TOK_TABLE_OR_COL x) value)) (= (. (TOK_TABLE_OR_COL x) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL x) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j2))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL res) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL res) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-7 depends on stages: Stage-9 - Stage-8 depends on stages: Stage-7 - Stage-6 depends on stages: Stage-8 + Stage-7 is a root stage + Stage-6 depends on stages: Stage-7 Stage-0 depends on stages: Stage-6 Stage-3 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - res:x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - res:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -2942,26 +2751,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work - - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 + Alias -> Map Local Tables: + res:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + res:x + TableScan + alias: x Stage: Stage-6 Map Reduce @@ -2999,6 +2796,14 @@ STAGE PLANS: name: default.dest_j2 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x Stage: Stage-0 Move Operator @@ -3187,34 +2992,12 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) res) (TOK_TABREF (TOK_TABNAME srcpart) y) (and (and (= (. (TOK_TABLE_OR_COL res) value) (. (TOK_TABLE_OR_COL y) value)) (= (. (TOK_TABLE_OR_COL y) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL y) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j2))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL res) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL res) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-7 depends on stages: Stage-9 - Stage-8 depends on stages: Stage-7 - Stage-6 depends on stages: Stage-8 + Stage-7 is a root stage + Stage-6 depends on stages: Stage-7 Stage-0 depends on stages: Stage-6 Stage-3 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - res:x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - res:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -3249,26 +3032,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work - - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 + Alias -> Map Local Tables: + res:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + res:x + TableScan + alias: x Stage: Stage-6 Map Reduce @@ -3306,6 +3077,14 @@ STAGE PLANS: name: default.dest_j2 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + y + TableScan + alias: y Stage: Stage-0 Move Operator diff --git ql/src/test/results/clientpositive/join33.q.out ql/src/test/results/clientpositive/join33.q.out index 9e84d84..729407d 100644 --- ql/src/test/results/clientpositive/join33.q.out +++ ql/src/test/results/clientpositive/join33.q.out @@ -23,95 +23,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL x) value) (. (TOK_TABLE_OR_COL z) value)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-5 depends on stages: Stage-7 + Stage-5 is a root stage Stage-0 depends on stages: Stage-5 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - z - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: hr=11 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - hr 11 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 0 - partition_columns ds/hr - rawDataSize 0 - serialization.ddl struct srcpart { string 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: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - Statistics: - numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - z - TableScan - alias: z - Statistics: - numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col5} {_col0} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -188,6 +104,68 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + z + Fetch Operator + limit: -1 + Partition Description: + Partition + base file name: hr=11 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + numFiles 1 + numRows 0 + partition_columns ds/hr + rawDataSize 0 + serialization.ddl struct srcpart { string 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: + bucket_count -1 + columns key,value + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + serialization.ddl struct srcpart { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart + name: default.srcpart + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + Statistics: + numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + z + TableScan + alias: z + Statistics: + numRows: 58 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/join34.q.out ql/src/test/results/clientpositive/join34.q.out index 1e2aa91..405b5e2 100644 --- ql/src/test/results/clientpositive/join34.q.out +++ ql/src/test/results/clientpositive/join34.q.out @@ -31,35 +31,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) key) 20)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) value) value)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) key) 100))))) subq1) (TOK_TABREF (TOK_TABNAME src1) x) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL subq1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) value))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-5 depends on stages: Stage-6 + Stage-5 is a root stage Stage-0 depends on stages: Stage-5 Stage-2 depends on stages: Stage-0 STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - Statistics: - numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col1} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -217,6 +193,17 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + Statistics: + numRows: 1 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/join35.q.out ql/src/test/results/clientpositive/join35.q.out index 83f5ba2..3150dba 100644 --- ql/src/test/results/clientpositive/join35.q.out +++ ql/src/test/results/clientpositive/join35.q.out @@ -32,9 +32,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-8 depends on stages: Stage-1, Stage-4 , consists of Stage-9, Stage-10, Stage-2 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 + Stage-8 depends on stages: Stage-1, Stage-4 , consists of Stage-6, Stage-10, Stage-2 + Stage-6 has a backup stage: Stage-2 Stage-0 depends on stages: Stage-2, Stage-6, Stage-7 Stage-3 depends on stages: Stage-0 Stage-10 has a backup stage: Stage-2 @@ -181,27 +180,6 @@ STAGE PLANS: Stage: Stage-8 Conditional Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {_col1} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -305,6 +283,15 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/join36.q.out ql/src/test/results/clientpositive/join36.q.out index 922ddef..5317a04 100644 --- ql/src/test/results/clientpositive/join36.q.out +++ ql/src/test/results/clientpositive/join36.q.out @@ -57,8 +57,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME tmp1) x) (TOK_TABREF (TOK_TABNAME tmp2) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST x))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) cnt))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -68,26 +67,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} {cnt} - 1 {cnt} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -125,6 +104,14 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x Stage: Stage-7 Conditional Operator diff --git ql/src/test/results/clientpositive/join37.q.out ql/src/test/results/clientpositive/join37.q.out index db905bd..2933f28 100644 --- ql/src/test/results/clientpositive/join37.q.out +++ ql/src/test/results/clientpositive/join37.q.out @@ -17,8 +17,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST X))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -28,26 +27,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -85,6 +64,14 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x Stage: Stage-7 Conditional Operator diff --git ql/src/test/results/clientpositive/join_map_ppr.q.out ql/src/test/results/clientpositive/join_map_ppr.q.out index 131e65c..3b3e440 100644 --- ql/src/test/results/clientpositive/join_map_ppr.q.out +++ ql/src/test/results/clientpositive/join_map_ppr.q.out @@ -21,8 +21,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME srcpart) z) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST x y))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08') (= (. (TOK_TABLE_OR_COL z) hr) 11))))) STAGE DEPENDENCIES: - Stage-10 is a root stage - Stage-1 depends on stages: Stage-10 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -32,51 +31,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - Statistics: - numRows: 2 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - y - TableScan - alias: y - Statistics: - numRows: 29 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -142,6 +96,26 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + Statistics: + numRows: 2 dataSize: 216 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + y + TableScan + alias: y + Statistics: + numRows: 29 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -576,8 +550,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1_copy) x) (TOK_TABREF (TOK_TABNAME src_copy) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME srcpart) z) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST x y))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value))) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08') (= (. (TOK_TABLE_OR_COL z) hr) 11))))) STAGE DEPENDENCIES: - Stage-10 is a root stage - Stage-1 depends on stages: Stage-10 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -587,51 +560,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - x - Fetch Operator - limit: -1 - y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - x - TableScan - alias: x - Statistics: - numRows: 25 dataSize: 191 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 2 - y - TableScan - alias: y - Statistics: - numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {value} - 2 {value} - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[key])] - 2 [GenericUDFBridge(Column[key])] - Position of Big Table: 2 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -702,6 +630,26 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + x + Fetch Operator + limit: -1 + y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + x + TableScan + alias: x + Statistics: + numRows: 25 dataSize: 191 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + y + TableScan + alias: y + Statistics: + numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/join_reorder4.q.out ql/src/test/results/clientpositive/join_reorder4.q.out index b14ea08..dfba3f6 100644 --- ql/src/test/results/clientpositive/join_reorder4.q.out +++ ql/src/test/results/clientpositive/join_reorder4.q.out @@ -39,50 +39,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) a) (TOK_TABREF (TOK_TABNAME T2) b) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key2))) (TOK_TABREF (TOK_TABNAME T3) c) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL c) key3)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_STREAMTABLE (TOK_HINTARGLIST a))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME a))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME b))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME c)))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-4 depends on stages: Stage-5 + Stage-4 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key1} {val1} - 1 {key2} {val2} - 2 {key3} {val3} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key2]] - 2 [Column[key3]] - Position of Big Table: 1 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 {key1} {val1} - 1 {key2} {val2} - 2 {key3} {val3} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key2]] - 2 [Column[key3]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -128,6 +88,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + c + TableScan + alias: c Stage: Stage-0 Fetch Operator @@ -154,50 +128,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) a) (TOK_TABREF (TOK_TABNAME T2) b) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key2))) (TOK_TABREF (TOK_TABNAME T3) c) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL c) key3)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_STREAMTABLE (TOK_HINTARGLIST b))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME a))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME b))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME c)))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-4 depends on stages: Stage-5 + Stage-4 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key1} {val1} - 1 {key2} {val2} - 2 {key3} {val3} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key2]] - 2 [Column[key3]] - Position of Big Table: 1 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 {key1} {val1} - 1 {key2} {val2} - 2 {key3} {val3} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key2]] - 2 [Column[key3]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -243,6 +177,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + c + TableScan + alias: c Stage: Stage-0 Fetch Operator @@ -269,50 +217,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) a) (TOK_TABREF (TOK_TABNAME T2) b) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL b) key2))) (TOK_TABREF (TOK_TABNAME T3) c) (= (. (TOK_TABLE_OR_COL a) key1) (. (TOK_TABLE_OR_COL c) key3)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_STREAMTABLE (TOK_HINTARGLIST c))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME a))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME b))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME c)))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-4 depends on stages: Stage-5 + Stage-4 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key1} {val1} - 1 {key2} {val2} - 2 {key3} {val3} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key2]] - 2 [Column[key3]] - Position of Big Table: 1 - c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 {key1} {val1} - 1 {key2} {val2} - 2 {key3} {val3} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key2]] - 2 [Column[key3]] - Position of Big Table: 1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -358,6 +266,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + c + TableScan + alias: c Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/join_star.q.out ql/src/test/results/clientpositive/join_star.q.out index 9b1adbf..d7a4f7b 100644 --- ql/src/test/results/clientpositive/join_star.q.out +++ ql/src/test/results/clientpositive/join_star.q.out @@ -94,31 +94,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME fact)) (TOK_TABREF (TOK_TABNAME dim1)) (= (. (TOK_TABLE_OR_COL fact) d1) (. (TOK_TABLE_OR_COL dim1) f1)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL m1)) (TOK_SELEXPR (TOK_TABLE_OR_COL m2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f2))))) STAGE DEPENDENCIES: - Stage-4 is a root stage - Stage-3 depends on stages: Stage-4 + Stage-3 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - dim1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - dim1 - TableScan - alias: dim1 - HashTable Sink Operator - condition expressions: - 0 {m1} {m2} - 1 {f2} - handleSkewJoin: false - keys: - 0 [Column[d1]] - 1 [Column[f1]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -155,6 +134,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + dim1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + dim1 + TableScan + alias: dim1 Stage: Stage-0 Fetch Operator @@ -188,46 +175,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME fact)) (TOK_TABREF (TOK_TABNAME dim1)) (= (. (TOK_TABLE_OR_COL fact) d1) (. (TOK_TABLE_OR_COL dim1) f1))) (TOK_TABREF (TOK_TABNAME dim2)) (= (. (TOK_TABLE_OR_COL fact) d2) (. (TOK_TABLE_OR_COL dim2) f3)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL m1)) (TOK_SELEXPR (TOK_TABLE_OR_COL m2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f4))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-4 depends on stages: Stage-6 + Stage-4 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - dim1 - Fetch Operator - limit: -1 - dim2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - dim1 - TableScan - alias: dim1 - HashTable Sink Operator - condition expressions: - 0 {m1} {m2} {d2} - 1 {f2} - handleSkewJoin: false - keys: - 0 [Column[d1]] - 1 [Column[f1]] - Position of Big Table: 0 - dim2 - TableScan - alias: dim2 - HashTable Sink Operator - condition expressions: - 0 {_col7} {_col0} {_col1} - 1 {f4} - handleSkewJoin: false - keys: - 0 [Column[_col3]] - 1 [Column[f3]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -278,6 +229,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + dim1 + Fetch Operator + limit: -1 + dim2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + dim1 + TableScan + alias: dim1 + dim2 + TableScan + alias: dim2 Stage: Stage-0 Fetch Operator @@ -313,46 +278,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME fact)) (TOK_TABREF (TOK_TABNAME dim1)) (= (. (TOK_TABLE_OR_COL fact) d1) (. (TOK_TABLE_OR_COL dim1) f1))) (TOK_TABREF (TOK_TABNAME dim2)) (= (. (TOK_TABLE_OR_COL dim1) f2) (. (TOK_TABLE_OR_COL dim2) f3)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL m1)) (TOK_SELEXPR (TOK_TABLE_OR_COL m2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f4))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-4 depends on stages: Stage-6 + Stage-4 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - dim1 - Fetch Operator - limit: -1 - dim2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - dim1 - TableScan - alias: dim1 - HashTable Sink Operator - condition expressions: - 0 {m1} {m2} - 1 {f2} - handleSkewJoin: false - keys: - 0 [Column[d1]] - 1 [Column[f1]] - Position of Big Table: 0 - dim2 - TableScan - alias: dim2 - HashTable Sink Operator - condition expressions: - 0 {_col7} {_col0} {_col1} - 1 {f4} - handleSkewJoin: false - keys: - 0 [Column[_col7]] - 1 [Column[f3]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -403,6 +332,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + dim1 + Fetch Operator + limit: -1 + dim2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + dim1 + TableScan + alias: dim1 + dim2 + TableScan + alias: dim2 Stage: Stage-0 Fetch Operator @@ -438,46 +381,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME fact)) (TOK_TABREF (TOK_TABNAME dim1)) (= (. (TOK_TABLE_OR_COL fact) d1) (. (TOK_TABLE_OR_COL dim1) f1))) (TOK_TABREF (TOK_TABNAME dim2)) (= (. (TOK_TABLE_OR_COL dim1) f2) (. (TOK_TABLE_OR_COL dim2) f3)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL m1)) (TOK_SELEXPR (TOK_TABLE_OR_COL m2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f4))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-4 depends on stages: Stage-6 + Stage-4 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - dim1 - Fetch Operator - limit: -1 - dim2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - dim1 - TableScan - alias: dim1 - HashTable Sink Operator - condition expressions: - 0 {m1} {m2} - 1 {f2} - handleSkewJoin: false - keys: - 0 [Column[d1]] - 1 [Column[f1]] - Position of Big Table: 0 - dim2 - TableScan - alias: dim2 - HashTable Sink Operator - condition expressions: - 0 {_col7} {_col0} {_col1} - 1 {f4} - handleSkewJoin: false - keys: - 0 [Column[_col7]] - 1 [Column[f3]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -528,6 +435,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + dim1 + Fetch Operator + limit: -1 + dim2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + dim1 + TableScan + alias: dim1 + dim2 + TableScan + alias: dim2 Stage: Stage-0 Fetch Operator @@ -579,125 +500,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME fact)) (TOK_TABREF (TOK_TABNAME dim1)) (= (. (TOK_TABLE_OR_COL fact) d1) (. (TOK_TABLE_OR_COL dim1) f1))) (TOK_TABREF (TOK_TABNAME dim2)) (= (. (TOK_TABLE_OR_COL dim1) f2) (. (TOK_TABLE_OR_COL dim2) f3))) (TOK_TABREF (TOK_TABNAME dim3)) (= (. (TOK_TABLE_OR_COL fact) d2) (. (TOK_TABLE_OR_COL dim3) f5))) (TOK_TABREF (TOK_TABNAME dim4)) (= (. (TOK_TABLE_OR_COL dim3) f6) (. (TOK_TABLE_OR_COL dim4) f7))) (TOK_TABREF (TOK_TABNAME dim5)) (= (. (TOK_TABLE_OR_COL dim4) f8) (. (TOK_TABLE_OR_COL dim5) f9))) (TOK_TABREF (TOK_TABNAME dim6)) (= (. (TOK_TABLE_OR_COL dim3) f6) (. (TOK_TABLE_OR_COL dim6) f11))) (TOK_TABREF (TOK_TABNAME dim7)) (= (. (TOK_TABLE_OR_COL dim6) f12) (. (TOK_TABLE_OR_COL dim7) f13)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL m1)) (TOK_SELEXPR (TOK_TABLE_OR_COL m2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f2)) (TOK_SELEXPR (TOK_TABLE_OR_COL f4)) (TOK_SELEXPR (TOK_TABLE_OR_COL f6)) (TOK_SELEXPR (TOK_TABLE_OR_COL f8)) (TOK_SELEXPR (TOK_TABLE_OR_COL f10)) (TOK_SELEXPR (TOK_TABLE_OR_COL f12)) (TOK_SELEXPR (TOK_TABLE_OR_COL f14))))) STAGE DEPENDENCIES: - Stage-18 is a root stage - Stage-12 depends on stages: Stage-18 + Stage-12 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-18 - Map Reduce Local Work - Alias -> Map Local Tables: - dim1 - Fetch Operator - limit: -1 - dim2 - Fetch Operator - limit: -1 - dim3 - Fetch Operator - limit: -1 - dim4 - Fetch Operator - limit: -1 - dim5 - Fetch Operator - limit: -1 - dim6 - Fetch Operator - limit: -1 - dim7 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - dim1 - TableScan - alias: dim1 - HashTable Sink Operator - condition expressions: - 0 {m1} {m2} {d2} - 1 {f2} - handleSkewJoin: false - keys: - 0 [Column[d1]] - 1 [Column[f1]] - Position of Big Table: 0 - dim2 - TableScan - alias: dim2 - HashTable Sink Operator - condition expressions: - 0 {_col7} {_col0} {_col1} {_col3} - 1 {f4} - handleSkewJoin: false - keys: - 0 [Column[_col7]] - 1 [Column[f3]] - Position of Big Table: 0 - dim3 - TableScan - alias: dim3 - HashTable Sink Operator - condition expressions: - 0 {_col11} {_col1} {_col4} {_col5} - 1 {f6} - handleSkewJoin: false - keys: - 0 [Column[_col7]] - 1 [Column[f5]] - Position of Big Table: 0 - dim4 - TableScan - alias: dim4 - HashTable Sink Operator - condition expressions: - 0 {_col15} {_col1} {_col5} {_col8} {_col9} - 1 {f8} - 2 {f12} - handleSkewJoin: false - keys: - 0 [Column[_col15]] - 1 [Column[f7]] - 2 [Column[f11]] - Position of Big Table: 0 - dim5 - TableScan - alias: dim5 - HashTable Sink Operator - condition expressions: - 0 {_col1} {_col5} {_col9} {_col12} {_col13} {_col23} {_col19} - 1 {f10} - handleSkewJoin: false - keys: - 0 [Column[_col19]] - 1 [Column[f9]] - Position of Big Table: 0 - dim6 - TableScan - alias: dim6 - HashTable Sink Operator - condition expressions: - 0 {_col15} {_col1} {_col5} {_col8} {_col9} - 1 {f8} - 2 {f12} - handleSkewJoin: false - keys: - 0 [Column[_col15]] - 1 [Column[f7]] - 2 [Column[f11]] - Position of Big Table: 0 - dim7 - TableScan - alias: dim7 - HashTable Sink Operator - condition expressions: - 0 {_col1} {_col5} {_col9} {_col12} {_col13} {_col19} {_col27} {_col23} - 1 {f14} - handleSkewJoin: false - keys: - 0 [Column[_col19]] - 1 [Column[f13]] - Position of Big Table: 0 - Stage: Stage-12 Map Reduce Alias -> Map Operator Tree: @@ -809,6 +615,50 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + dim1 + Fetch Operator + limit: -1 + dim2 + Fetch Operator + limit: -1 + dim3 + Fetch Operator + limit: -1 + dim4 + Fetch Operator + limit: -1 + dim5 + Fetch Operator + limit: -1 + dim6 + Fetch Operator + limit: -1 + dim7 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + dim1 + TableScan + alias: dim1 + dim2 + TableScan + alias: dim2 + dim3 + TableScan + alias: dim3 + dim4 + TableScan + alias: dim4 + dim5 + TableScan + alias: dim5 + dim6 + TableScan + alias: dim6 + dim7 + TableScan + alias: dim7 Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/mapjoin1.q.out ql/src/test/results/clientpositive/mapjoin1.q.out index e8b4fa8..dd85635 100644 --- ql/src/test/results/clientpositive/mapjoin1.q.out +++ ql/src/test/results/clientpositive/mapjoin1.q.out @@ -33,34 +33,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_TABREF (TOK_TABNAME src) b) (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) true))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_LIMIT 10))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - filter predicates: - 0 - 1 {true} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -103,6 +79,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Stage: Stage-0 Fetch Operator @@ -138,34 +122,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_TABREF (TOK_TABNAME src) b) (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (< (* (. (TOK_TABLE_OR_COL b) key) 10) '1000')))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_LIMIT 10))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - filter predicates: - 0 - 1 {((key * 10) < '1000')} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -208,6 +168,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Stage: Stage-0 Fetch Operator @@ -245,34 +213,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION named_struct 'key' (TOK_TABLE_OR_COL key) 'value' (TOK_TABLE_OR_COL value)) kv)))) b) (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (> (. (. (TOK_TABLE_OR_COL b) kv) key) 200)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_LIMIT 10))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {_col0} {_col1} - filter predicates: - 0 - 1 {(_col1.key > 200)} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -322,6 +266,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Stage: Stage-0 Fetch Operator @@ -569,31 +521,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION named_struct 'key' (TOK_TABLE_OR_COL key) 'value' (TOK_TABLE_OR_COL value)) kv)))) b) (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (> (. (. (TOK_TABLE_OR_COL b) kv) key) 200)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_LIMIT 10))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {_col0} {_col1} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -644,6 +575,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/mapjoin_distinct.q.out ql/src/test/results/clientpositive/mapjoin_distinct.q.out index c4dc9d2..7bc5b9d 100644 --- ql/src/test/results/clientpositive/mapjoin_distinct.q.out +++ ql/src/test/results/clientpositive/mapjoin_distinct.q.out @@ -14,32 +14,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcpart) c) (TOK_TABREF (TOK_TABNAME srcpart) d) (AND (AND (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL d) key)) (= (. (TOK_TABLE_OR_COL c) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL d) ds) '2008-04-08')))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECTDI (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST d))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL c) value))))) STAGE DEPENDENCIES: - Stage-4 is a root stage - Stage-1 depends on stages: Stage-4 + Stage-1 is a root stage Stage-3 depends on stages: Stage-1 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - d - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - d - TableScan - alias: d - HashTable Sink Operator - condition expressions: - 0 {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -81,6 +60,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + d + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + d + TableScan + alias: d Reduce Operator Tree: Group By Operator bucketGroup: false @@ -180,31 +167,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcpart) c) (TOK_TABREF (TOK_TABNAME srcpart) d) (AND (AND (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL d) key)) (= (. (TOK_TABLE_OR_COL c) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL d) ds) '2008-04-08')))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECTDI (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST d))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL c) value))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - d - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - d - TableScan - alias: d - HashTable Sink Operator - condition expressions: - 0 {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -246,6 +212,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + d + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + d + TableScan + alias: d Reduce Operator Tree: Group By Operator bucketGroup: false @@ -315,32 +289,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcpart) c) (TOK_TABREF (TOK_TABNAME srcpart) d) (AND (AND (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL d) key)) (= (. (TOK_TABLE_OR_COL c) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL d) ds) '2008-04-08')))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECTDI (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST d))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL c) value))))) STAGE DEPENDENCIES: - Stage-4 is a root stage - Stage-1 depends on stages: Stage-4 + Stage-1 is a root stage Stage-3 depends on stages: Stage-1 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - d - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - d - TableScan - alias: d - HashTable Sink Operator - condition expressions: - 0 {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -375,6 +328,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + d + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + d + TableScan + alias: d Reduce Operator Tree: Group By Operator bucketGroup: false @@ -474,31 +435,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcpart) c) (TOK_TABREF (TOK_TABNAME srcpart) d) (AND (AND (= (. (TOK_TABLE_OR_COL c) key) (. (TOK_TABLE_OR_COL d) key)) (= (. (TOK_TABLE_OR_COL c) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL d) ds) '2008-04-08')))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECTDI (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST d))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL c) value))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - d - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - d - TableScan - alias: d - HashTable Sink Operator - condition expressions: - 0 {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -533,6 +473,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + d + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + d + TableScan + alias: d Reduce Operator Tree: Group By Operator bucketGroup: false diff --git ql/src/test/results/clientpositive/mapjoin_hook.q.out ql/src/test/results/clientpositive/mapjoin_hook.q.out index 8da5168..594ac9c 100644 --- ql/src/test/results/clientpositive/mapjoin_hook.q.out +++ ql/src/test/results/clientpositive/mapjoin_hook.q.out @@ -8,13 +8,13 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Input: default@src1 PREHOOK: Output: default@dest1 -[MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: 0 CONVERTED_MAPJOIN: 0 LOCAL_MAPJOIN: 1 COMMON_JOIN: 0 BACKUP_COMMON_JOIN: 0 +[MapJoinCounter PostHook] COMMON_JOIN: 0 HINTED_MAPJOIN: 1 HINTED_MAPJOIN_LOCAL: 0 CONVERTED_MAPJOIN: 0 CONVERTED_MAPJOIN_LOCAL: 0 BACKUP_COMMON_JOIN: 0 PREHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: 0 CONVERTED_MAPJOIN: 0 LOCAL_MAPJOIN: 1 COMMON_JOIN: 0 BACKUP_COMMON_JOIN: 0 +[MapJoinCounter PostHook] COMMON_JOIN: 0 HINTED_MAPJOIN: 1 HINTED_MAPJOIN_LOCAL: 0 CONVERTED_MAPJOIN: 0 CONVERTED_MAPJOIN_LOCAL: 0 BACKUP_COMMON_JOIN: 0 PREHOOK: query: FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value where (src1.ds = '2008-04-08' or src1.ds = '2008-04-09' )and (src1.hr = '12' or src1.hr = '11') @@ -26,46 +26,46 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@dest1 -Execution failed with exit status: 3 +Execution failed with exit status: 2 Obtaining error information Task failed! Task ID: - Stage-7 + Stage-4 Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask -[MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: 1 CONVERTED_MAPJOIN: 0 LOCAL_MAPJOIN: 0 COMMON_JOIN: 0 BACKUP_COMMON_JOIN: 1 +[MapJoinCounter PostHook] COMMON_JOIN: 0 HINTED_MAPJOIN: 0 HINTED_MAPJOIN_LOCAL: 0 CONVERTED_MAPJOIN: 1 CONVERTED_MAPJOIN_LOCAL: 0 BACKUP_COMMON_JOIN: 1 PREHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -Execution failed with exit status: 3 +Execution failed with exit status: 2 Obtaining error information Task failed! Task ID: - Stage-15 + Stage-10 Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask -Execution failed with exit status: 3 +Execution failed with exit status: 2 Obtaining error information Task failed! Task ID: - Stage-12 + Stage-6 Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask -[MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: 2 CONVERTED_MAPJOIN: 0 LOCAL_MAPJOIN: 0 COMMON_JOIN: 0 BACKUP_COMMON_JOIN: 2 +[MapJoinCounter PostHook] COMMON_JOIN: 0 HINTED_MAPJOIN: 0 HINTED_MAPJOIN_LOCAL: 0 CONVERTED_MAPJOIN: 2 CONVERTED_MAPJOIN_LOCAL: 0 BACKUP_COMMON_JOIN: 2 diff --git ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out index 2c9f7f6..a2bd45a 100644 --- ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out +++ ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out @@ -10,46 +10,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcpart)) (TOK_TABREF (TOK_TABNAME src)) (= (. (TOK_TABLE_OR_COL srcpart) value) (. (TOK_TABLE_OR_COL src) value))) (TOK_TABREF (TOK_TABNAME src1)) (= (. (TOK_TABLE_OR_COL srcpart) key) (. (TOK_TABLE_OR_COL src1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL srcpart) key))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-5 depends on stages: Stage-7 + Stage-5 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - src - Fetch Operator - limit: -1 - src1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src - TableScan - alias: src - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[value]] - 1 [Column[value]] - Position of Big Table: 0 - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -94,6 +58,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src + Fetch Operator + limit: -1 + src1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src + TableScan + alias: src + src1 + TableScan + alias: src1 Stage: Stage-0 Fetch Operator @@ -107,46 +85,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcpart)) (TOK_TABREF (TOK_TABNAME src)) (= (. (TOK_TABLE_OR_COL srcpart) value) (. (TOK_TABLE_OR_COL src) value))) (TOK_TABREF (TOK_TABNAME src1)) (= (. (TOK_TABLE_OR_COL srcpart) key) (. (TOK_TABLE_OR_COL src1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_GROUPBY (TOK_TABLE_OR_COL ds)))) STAGE DEPENDENCIES: - Stage-8 is a root stage - Stage-3 depends on stages: Stage-8 + Stage-3 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - src - Fetch Operator - limit: -1 - src1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src - TableScan - alias: src - HashTable Sink Operator - condition expressions: - 0 {key} {ds} - 1 - handleSkewJoin: false - keys: - 0 [Column[value]] - 1 [Column[value]] - Position of Big Table: 0 - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {_col2} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -205,6 +147,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src + Fetch Operator + limit: -1 + src1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src + TableScan + alias: src + src1 + TableScan + alias: src1 Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/mapjoin_subquery.q.out ql/src/test/results/clientpositive/mapjoin_subquery.q.out index e90a50a..e3a8fca 100644 --- ql/src/test/results/clientpositive/mapjoin_subquery.q.out +++ ql/src/test/results/clientpositive/mapjoin_subquery.q.out @@ -20,46 +20,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value2)))) subq) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL subq) key1) (. (TOK_TABLE_OR_COL z) key)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-5 depends on stages: Stage-7 + Stage-5 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - subq:x - Fetch Operator - limit: -1 - z - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - z - TableScan - alias: z - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -111,6 +75,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq:x + Fetch Operator + limit: -1 + z + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq:x + TableScan + alias: x + z + TableScan + alias: z Stage: Stage-0 Fetch Operator @@ -267,46 +245,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value2)))) subq) (TOK_TABREF (TOK_TABNAME srcpart) z) (and (and (= (. (TOK_TABLE_OR_COL subq) key1) (. (TOK_TABLE_OR_COL z) key)) (= (. (TOK_TABLE_OR_COL z) ds) '2008-04-08')) (= (. (TOK_TABLE_OR_COL z) hr) 11)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq) key1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL z) value))))) STAGE DEPENDENCIES: - Stage-8 is a root stage - Stage-3 depends on stages: Stage-8 + Stage-3 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - subq:x - Fetch Operator - limit: -1 - z - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq:x - TableScan - alias: x - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - z - TableScan - alias: z - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -364,6 +306,20 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq:x + Fetch Operator + limit: -1 + z + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq:x + TableScan + alias: x + z + TableScan + alias: z Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/mapjoin_subquery2.q.out ql/src/test/results/clientpositive/mapjoin_subquery2.q.out index 220bf322..70e058c 100644 --- ql/src/test/results/clientpositive/mapjoin_subquery2.q.out +++ ql/src/test/results/clientpositive/mapjoin_subquery2.q.out @@ -71,46 +71,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME y)) (TOK_TABREF (TOK_TABNAME x)) (= (. (TOK_TABLE_OR_COL x) id) (. (TOK_TABLE_OR_COL y) id)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) id) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) name) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) id) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) name) value2)))) subq) (TOK_TABREF (TOK_TABNAME z)) (= (. (TOK_TABLE_OR_COL subq) key1) (. (TOK_TABLE_OR_COL z) id)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq) value1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq) value2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) id)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) name))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-5 depends on stages: Stage-7 + Stage-5 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - subq:y - Fetch Operator - limit: -1 - z - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - subq:y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 {id} {name} - 1 {name} {id} - handleSkewJoin: false - keys: - 0 [Column[id]] - 1 [Column[id]] - Position of Big Table: 1 - z - TableScan - alias: z - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} - 1 {id} {name} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[id]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -176,6 +140,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + subq:y + Fetch Operator + limit: -1 + z + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq:y + TableScan + alias: y + z + TableScan + alias: z Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/mapjoin_test_outer.q.out ql/src/test/results/clientpositive/mapjoin_test_outer.q.out index d66e079..525650e 100644 --- ql/src/test/results/clientpositive/mapjoin_test_outer.q.out +++ ql/src/test/results/clientpositive/mapjoin_test_outer.q.out @@ -263,50 +263,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1)) (TOK_TABREF (TOK_TABNAME dest_1) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key))) (TOK_TABREF (TOK_TABNAME dest_2) src3) (= (. (TOK_TABLE_OR_COL src2) key) (. (TOK_TABLE_OR_COL src3) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST src1 src2))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src1) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src1) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src2) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src2) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src3) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src3) value))))) STAGE DEPENDENCIES: - Stage-4 is a root stage - Stage-1 depends on stages: Stage-4 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 - src2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - src2 - TableScan - alias: src2 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 2 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -374,6 +334,20 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + src2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 + src2 + TableScan + alias: src2 Reduce Operator Tree: Extract File Output Operator @@ -1163,50 +1137,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1)) (TOK_TABREF (TOK_TABNAME dest_1) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key))) (TOK_TABREF (TOK_TABNAME dest_2) src3) (= (. (TOK_TABLE_OR_COL src2) key) (. (TOK_TABLE_OR_COL src3) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src1) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src1) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src2) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src2) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src3) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src3) value))))) STAGE DEPENDENCIES: - Stage-6 is a root stage - Stage-2 depends on stages: Stage-6 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - src1 - Fetch Operator - limit: -1 - src3 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - src1 - TableScan - alias: src1 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 1 - src3 - TableScan - alias: src3 - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - 2 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - 2 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -1274,6 +1208,20 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + src1 + Fetch Operator + limit: -1 + src3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + src1 + TableScan + alias: src1 + src3 + TableScan + alias: src3 Reduce Operator Tree: Extract File Output Operator diff --git ql/src/test/results/clientpositive/multiMapJoin1.q.out ql/src/test/results/clientpositive/multiMapJoin1.q.out index 624b47f..1899cd3 100644 --- ql/src/test/results/clientpositive/multiMapJoin1.q.out +++ ql/src/test/results/clientpositive/multiMapJoin1.q.out @@ -189,46 +189,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bigTbl)) (TOK_TABREF (TOK_TABNAME smallTbl1)) (= (. (TOK_TABLE_OR_COL bigTbl) key) (. (TOK_TABLE_OR_COL smallTbl1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value2)))) firstjoin) (TOK_TABREF (TOK_TABNAME smallTbl2)) (= (. (TOK_TABLE_OR_COL firstjoin) value1) (. (TOK_TABLE_OR_COL smallTbl2) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-2 depends on stages: Stage-7 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - firstjoin:smalltbl1 - Fetch Operator - limit: -1 - smalltbl2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - firstjoin:smalltbl1 - TableScan - alias: smalltbl1 - HashTable Sink Operator - condition expressions: - 0 {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - smalltbl2 - TableScan - alias: smalltbl2 - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -278,6 +242,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + firstjoin:smalltbl1 + Fetch Operator + limit: -1 + smalltbl2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + firstjoin:smalltbl1 + TableScan + alias: smalltbl1 + smalltbl2 + TableScan + alias: smalltbl2 Reduce Operator Tree: Group By Operator aggregations: @@ -373,46 +351,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bigTbl)) (TOK_TABREF (TOK_TABNAME smallTbl1)) (= (. (TOK_TABLE_OR_COL bigTbl) key) (. (TOK_TABLE_OR_COL smallTbl1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value2)))) firstjoin) (TOK_TABREF (TOK_TABNAME smallTbl2)) (= (. (TOK_TABLE_OR_COL firstjoin) value1) (. (TOK_TABLE_OR_COL smallTbl2) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-2 depends on stages: Stage-7 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - firstjoin:smalltbl1 - Fetch Operator - limit: -1 - smalltbl2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - firstjoin:smalltbl1 - TableScan - alias: smalltbl1 - HashTable Sink Operator - condition expressions: - 0 {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - smalltbl2 - TableScan - alias: smalltbl2 - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -462,6 +404,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + firstjoin:smalltbl1 + Fetch Operator + limit: -1 + smalltbl2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + firstjoin:smalltbl1 + TableScan + alias: smalltbl1 + smalltbl2 + TableScan + alias: smalltbl2 Reduce Operator Tree: Group By Operator aggregations: @@ -561,46 +517,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bigTbl)) (TOK_TABREF (TOK_TABNAME smallTbl1)) (= (. (TOK_TABLE_OR_COL bigTbl) key) (. (TOK_TABLE_OR_COL smallTbl1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value2)))) firstjoin) (TOK_TABREF (TOK_TABNAME smallTbl2)) (= (. (TOK_TABLE_OR_COL firstjoin) value1) (. (TOK_TABLE_OR_COL smallTbl2) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL smallTbl2) key)))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-2 depends on stages: Stage-7 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - firstjoin:smalltbl1 - Fetch Operator - limit: -1 - smalltbl2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - firstjoin:smalltbl1 - TableScan - alias: smalltbl1 - HashTable Sink Operator - condition expressions: - 0 {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - smalltbl2 - TableScan - alias: smalltbl2 - HashTable Sink Operator - condition expressions: - 0 - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -664,6 +584,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + firstjoin:smalltbl1 + Fetch Operator + limit: -1 + smalltbl2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + firstjoin:smalltbl1 + TableScan + alias: smalltbl1 + smalltbl2 + TableScan + alias: smalltbl2 Reduce Operator Tree: Group By Operator aggregations: @@ -916,30 +850,22 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bigTbl)) (TOK_TABREF (TOK_TABNAME smallTbl1)) (= (. (TOK_TABLE_OR_COL bigTbl) key1) (. (TOK_TABLE_OR_COL smallTbl1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl1) key) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value2)))) join1) (TOK_TABREF (TOK_TABNAME smallTbl2)) (= (. (TOK_TABLE_OR_COL join1) value1) (. (TOK_TABLE_OR_COL smallTbl2) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key3) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl2) key) key4) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) value1) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) value2) value2)))) join2) (TOK_TABREF (TOK_TABNAME smallTbl3)) (= (. (TOK_TABLE_OR_COL join2) key2) (. (TOK_TABLE_OR_COL smallTbl3) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key3) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key4) key4) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl3) key) key5) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) value1) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) value2) value2)))) join3) (TOK_TABREF (TOK_TABNAME smallTbl4)) (= (. (TOK_TABLE_OR_COL join3) key3) (. (TOK_TABLE_OR_COL smallTbl4) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key3)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key4)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key5)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL smallTbl4) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) value1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) value2))))))) STAGE DEPENDENCIES: - Stage-20 is a root stage , consists of Stage-27, Stage-28, Stage-5 - Stage-27 has a backup stage: Stage-5 - Stage-18 depends on stages: Stage-27 - Stage-17 depends on stages: Stage-5, Stage-18, Stage-19 , consists of Stage-25, Stage-26, Stage-1 - Stage-25 has a backup stage: Stage-1 - Stage-15 depends on stages: Stage-25 - Stage-14 depends on stages: Stage-1, Stage-15, Stage-16 , consists of Stage-23, Stage-24, Stage-2 - Stage-23 has a backup stage: Stage-2 - Stage-12 depends on stages: Stage-23 - Stage-11 depends on stages: Stage-2, Stage-12, Stage-13 , consists of Stage-21, Stage-22, Stage-3 - Stage-21 has a backup stage: Stage-3 - Stage-9 depends on stages: Stage-21 + Stage-20 is a root stage , consists of Stage-18, Stage-19, Stage-5 + Stage-18 has a backup stage: Stage-5 + Stage-17 depends on stages: Stage-5, Stage-18, Stage-19 , consists of Stage-15, Stage-16, Stage-1 + Stage-15 has a backup stage: Stage-1 + Stage-14 depends on stages: Stage-1, Stage-15, Stage-16 , consists of Stage-12, Stage-13, Stage-2 + Stage-12 has a backup stage: Stage-2 + Stage-11 depends on stages: Stage-2, Stage-12, Stage-13 , consists of Stage-9, Stage-10, Stage-3 + Stage-9 has a backup stage: Stage-3 Stage-4 depends on stages: Stage-3, Stage-9, Stage-10 - Stage-22 has a backup stage: Stage-3 - Stage-10 depends on stages: Stage-22 + Stage-10 has a backup stage: Stage-3 Stage-3 - Stage-24 has a backup stage: Stage-2 - Stage-13 depends on stages: Stage-24 + Stage-13 has a backup stage: Stage-2 Stage-2 - Stage-26 has a backup stage: Stage-1 - Stage-16 depends on stages: Stage-26 + Stage-16 has a backup stage: Stage-1 Stage-1 - Stage-28 has a backup stage: Stage-5 - Stage-19 depends on stages: Stage-28 + Stage-19 has a backup stage: Stage-5 Stage-5 Stage-0 is a root stage @@ -947,26 +873,6 @@ STAGE PLANS: Stage: Stage-20 Conditional Operator - Stage: Stage-27 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:join1:smalltbl1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:join1:smalltbl1 - TableScan - alias: smalltbl1 - HashTable Sink Operator - condition expressions: - 0 {key1} {key2} {value} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-18 Map Reduce Alias -> Map Operator Tree: @@ -1007,30 +913,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:join1:smalltbl1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:join1:smalltbl1 + TableScan + alias: smalltbl1 Stage: Stage-17 Conditional Operator - Stage: Stage-25 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:smalltbl2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:smalltbl2 - TableScan - alias: smalltbl2 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col3]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-15 Map Reduce Alias -> Map Operator Tree: @@ -1072,30 +966,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:smalltbl2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:smalltbl2 + TableScan + alias: smalltbl2 Stage: Stage-14 Conditional Operator - Stage: Stage-23 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:smalltbl3 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:smalltbl3 - TableScan - alias: smalltbl3 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-12 Map Reduce Alias -> Map Operator Tree: @@ -1139,30 +1021,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:smalltbl3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:smalltbl3 + TableScan + alias: smalltbl3 Stage: Stage-11 Conditional Operator - Stage: Stage-21 - Map Reduce Local Work - Alias -> Map Local Tables: - smalltbl4 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - smalltbl4 - TableScan - alias: smalltbl4 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} {_col6} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col2]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-9 Map Reduce Alias -> Map Operator Tree: @@ -1221,6 +1091,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + smalltbl4 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + smalltbl4 + TableScan + alias: smalltbl4 Stage: Stage-4 Map Reduce @@ -1288,25 +1166,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-22 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} {_col6} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col2]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-10 Map Reduce Alias -> Map Operator Tree: @@ -1366,6 +1225,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-3 Map Reduce @@ -1460,25 +1326,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-24 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:$INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:$INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-13 Map Reduce Alias -> Map Operator Tree: @@ -1523,6 +1370,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:$INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:$INTNAME + TableScan Stage: Stage-2 Map Reduce @@ -1600,25 +1454,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-26 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:$INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:$INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col3]] - 1 [Column[value]] - Position of Big Table: 1 - Stage: Stage-16 Map Reduce Alias -> Map Operator Tree: @@ -1661,6 +1496,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:$INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:$INTNAME + TableScan Stage: Stage-1 Map Reduce @@ -1734,26 +1576,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-28 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:join1:bigtbl - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:join1:bigtbl - TableScan - alias: bigtbl - HashTable Sink Operator - condition expressions: - 0 {key1} {key2} {value} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-19 Map Reduce Alias -> Map Operator Tree: @@ -1794,6 +1616,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:join1:bigtbl + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:join1:bigtbl + TableScan + alias: bigtbl Stage: Stage-5 Map Reduce @@ -2045,76 +1875,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bigTbl)) (TOK_TABREF (TOK_TABNAME smallTbl1)) (= (. (TOK_TABLE_OR_COL bigTbl) key1) (. (TOK_TABLE_OR_COL smallTbl1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl1) key) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value2)))) join1) (TOK_TABREF (TOK_TABNAME smallTbl2)) (= (. (TOK_TABLE_OR_COL join1) value1) (. (TOK_TABLE_OR_COL smallTbl2) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key3) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl2) key) key4) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) value1) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) value2) value2)))) join2) (TOK_TABREF (TOK_TABNAME smallTbl3)) (= (. (TOK_TABLE_OR_COL join2) key2) (. (TOK_TABLE_OR_COL smallTbl3) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key3) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key4) key4) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl3) key) key5) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) value1) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) value2) value2)))) join3) (TOK_TABREF (TOK_TABNAME smallTbl4)) (= (. (TOK_TABLE_OR_COL join3) key3) (. (TOK_TABLE_OR_COL smallTbl4) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key3)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key4)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key5)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL smallTbl4) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) value1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) value2))))))) STAGE DEPENDENCIES: - Stage-13 is a root stage - Stage-4 depends on stages: Stage-13 + Stage-4 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-13 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:join1:smalltbl1 - Fetch Operator - limit: -1 - join3:join2:smalltbl2 - Fetch Operator - limit: -1 - join3:smalltbl3 - Fetch Operator - limit: -1 - smalltbl4 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:join1:smalltbl1 - TableScan - alias: smalltbl1 - HashTable Sink Operator - condition expressions: - 0 {key1} {key2} {value} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key]] - Position of Big Table: 0 - join3:join2:smalltbl2 - TableScan - alias: smalltbl2 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col3]] - 1 [Column[value]] - Position of Big Table: 0 - join3:smalltbl3 - TableScan - alias: smalltbl3 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[key]] - Position of Big Table: 0 - smalltbl4 - TableScan - alias: smalltbl4 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} {_col6} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col2]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -2268,6 +2032,32 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:join1:smalltbl1 + Fetch Operator + limit: -1 + join3:join2:smalltbl2 + Fetch Operator + limit: -1 + join3:smalltbl3 + Fetch Operator + limit: -1 + smalltbl4 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:join1:smalltbl1 + TableScan + alias: smalltbl1 + join3:join2:smalltbl2 + TableScan + alias: smalltbl2 + join3:smalltbl3 + TableScan + alias: smalltbl3 + smalltbl4 + TableScan + alias: smalltbl4 Reduce Operator Tree: Group By Operator aggregations: @@ -2494,48 +2284,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bigTbl)) (TOK_TABREF (TOK_TABNAME smallTbl1)) (= (. (TOK_TABLE_OR_COL bigTbl) key1) (. (TOK_TABLE_OR_COL smallTbl1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl1) key) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value2)))) join1) (TOK_TABREF (TOK_TABNAME smallTbl2)) (= (. (TOK_TABLE_OR_COL join1) value1) (. (TOK_TABLE_OR_COL smallTbl2) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key3) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl2) key) key4) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) value1) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) value2) value2)))) join2) (TOK_TABREF (TOK_TABNAME smallTbl3)) (= (. (TOK_TABLE_OR_COL join2) key2) (. (TOK_TABLE_OR_COL smallTbl3) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key3) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key4) key4) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl3) key) key5) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) value1) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) value2) value2)))) join3) (TOK_TABREF (TOK_TABNAME smallTbl4)) (= (. (TOK_TABLE_OR_COL join3) key3) (. (TOK_TABLE_OR_COL smallTbl4) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key3)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key4)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key5)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL smallTbl4) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) value1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) value2))))))) STAGE DEPENDENCIES: - Stage-14 is a root stage - Stage-11 depends on stages: Stage-14 - Stage-13 depends on stages: Stage-11 - Stage-4 depends on stages: Stage-13 + Stage-11 is a root stage + Stage-4 depends on stages: Stage-11 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-14 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:join1:smalltbl1 - Fetch Operator - limit: -1 - join3:join2:smalltbl2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:join1:smalltbl1 - TableScan - alias: smalltbl1 - HashTable Sink Operator - condition expressions: - 0 {key1} {key2} {value} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key]] - Position of Big Table: 0 - join3:join2:smalltbl2 - TableScan - alias: smalltbl2 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col3]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-11 Map Reduce Alias -> Map Operator Tree: @@ -2603,41 +2356,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work - - Stage: Stage-13 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:smalltbl3 - Fetch Operator - limit: -1 - smalltbl4 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:smalltbl3 - TableScan - alias: smalltbl3 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[key]] - Position of Big Table: 0 - smalltbl4 - TableScan - alias: smalltbl4 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} {_col6} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col2]] - 1 [Column[key]] - Position of Big Table: 0 + Alias -> Map Local Tables: + join3:join2:join1:smalltbl1 + Fetch Operator + limit: -1 + join3:join2:smalltbl2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:join1:smalltbl1 + TableScan + alias: smalltbl1 + join3:join2:smalltbl2 + TableScan + alias: smalltbl2 Stage: Stage-4 Map Reduce @@ -2739,6 +2471,20 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:smalltbl3 + Fetch Operator + limit: -1 + smalltbl4 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:smalltbl3 + TableScan + alias: smalltbl3 + smalltbl4 + TableScan + alias: smalltbl4 Reduce Operator Tree: Group By Operator aggregations: @@ -2961,30 +2707,22 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bigTbl)) (TOK_TABREF (TOK_TABNAME smallTbl1)) (= (. (TOK_TABLE_OR_COL bigTbl) key1) (. (TOK_TABLE_OR_COL smallTbl1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl1) key) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL bigTbl) value) value2)))) join1) (TOK_TABREF (TOK_TABNAME smallTbl2)) (= (. (TOK_TABLE_OR_COL join1) value1) (. (TOK_TABLE_OR_COL smallTbl2) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) key3) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl2) key) key4) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) value1) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join1) value2) value2)))) join2) (TOK_TABREF (TOK_TABNAME smallTbl3)) (= (. (TOK_TABLE_OR_COL join2) key2) (. (TOK_TABLE_OR_COL smallTbl3) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key2) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key3) key3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) key4) key4) (TOK_SELEXPR (. (TOK_TABLE_OR_COL smallTbl3) key) key5) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) value1) value1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL join2) value2) value2)))) join3) (TOK_TABREF (TOK_TABNAME smallTbl4)) (= (. (TOK_TABLE_OR_COL join3) key3) (. (TOK_TABLE_OR_COL smallTbl4) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key3)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key4)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) key5)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL smallTbl4) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) value1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL join3) value2))))))) STAGE DEPENDENCIES: - Stage-20 is a root stage , consists of Stage-27, Stage-28, Stage-5 - Stage-27 has a backup stage: Stage-5 - Stage-18 depends on stages: Stage-27 - Stage-17 depends on stages: Stage-5, Stage-18, Stage-19 , consists of Stage-25, Stage-26, Stage-1 - Stage-25 has a backup stage: Stage-1 - Stage-15 depends on stages: Stage-25 - Stage-14 depends on stages: Stage-1, Stage-15, Stage-16 , consists of Stage-23, Stage-24, Stage-2 - Stage-23 has a backup stage: Stage-2 - Stage-12 depends on stages: Stage-23 - Stage-11 depends on stages: Stage-2, Stage-12, Stage-13 , consists of Stage-21, Stage-22, Stage-3 - Stage-21 has a backup stage: Stage-3 - Stage-9 depends on stages: Stage-21 + Stage-20 is a root stage , consists of Stage-18, Stage-19, Stage-5 + Stage-18 has a backup stage: Stage-5 + Stage-17 depends on stages: Stage-5, Stage-18, Stage-19 , consists of Stage-15, Stage-16, Stage-1 + Stage-15 has a backup stage: Stage-1 + Stage-14 depends on stages: Stage-1, Stage-15, Stage-16 , consists of Stage-12, Stage-13, Stage-2 + Stage-12 has a backup stage: Stage-2 + Stage-11 depends on stages: Stage-2, Stage-12, Stage-13 , consists of Stage-9, Stage-10, Stage-3 + Stage-9 has a backup stage: Stage-3 Stage-4 depends on stages: Stage-3, Stage-9, Stage-10 - Stage-22 has a backup stage: Stage-3 - Stage-10 depends on stages: Stage-22 + Stage-10 has a backup stage: Stage-3 Stage-3 - Stage-24 has a backup stage: Stage-2 - Stage-13 depends on stages: Stage-24 + Stage-13 has a backup stage: Stage-2 Stage-2 - Stage-26 has a backup stage: Stage-1 - Stage-16 depends on stages: Stage-26 + Stage-16 has a backup stage: Stage-1 Stage-1 - Stage-28 has a backup stage: Stage-5 - Stage-19 depends on stages: Stage-28 + Stage-19 has a backup stage: Stage-5 Stage-5 Stage-0 is a root stage @@ -2992,26 +2730,6 @@ STAGE PLANS: Stage: Stage-20 Conditional Operator - Stage: Stage-27 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:join1:smalltbl1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:join1:smalltbl1 - TableScan - alias: smalltbl1 - HashTable Sink Operator - condition expressions: - 0 {key1} {key2} {value} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-18 Map Reduce Alias -> Map Operator Tree: @@ -3052,30 +2770,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:join1:smalltbl1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:join1:smalltbl1 + TableScan + alias: smalltbl1 Stage: Stage-17 Conditional Operator - Stage: Stage-25 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:smalltbl2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:smalltbl2 - TableScan - alias: smalltbl2 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col3]] - 1 [Column[value]] - Position of Big Table: 0 - Stage: Stage-15 Map Reduce Alias -> Map Operator Tree: @@ -3117,30 +2823,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:smalltbl2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:smalltbl2 + TableScan + alias: smalltbl2 Stage: Stage-14 Conditional Operator - Stage: Stage-23 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:smalltbl3 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:smalltbl3 - TableScan - alias: smalltbl3 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-12 Map Reduce Alias -> Map Operator Tree: @@ -3184,30 +2878,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:smalltbl3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:smalltbl3 + TableScan + alias: smalltbl3 Stage: Stage-11 Conditional Operator - Stage: Stage-21 - Map Reduce Local Work - Alias -> Map Local Tables: - smalltbl4 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - smalltbl4 - TableScan - alias: smalltbl4 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} {_col6} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col2]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-9 Map Reduce Alias -> Map Operator Tree: @@ -3266,6 +2948,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + smalltbl4 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + smalltbl4 + TableScan + alias: smalltbl4 Stage: Stage-4 Map Reduce @@ -3333,25 +3023,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-22 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} {_col6} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col2]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-10 Map Reduce Alias -> Map Operator Tree: @@ -3411,6 +3082,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-3 Map Reduce @@ -3505,25 +3183,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-24 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:$INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:$INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} {_col5} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-13 Map Reduce Alias -> Map Operator Tree: @@ -3568,6 +3227,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:$INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:$INTNAME + TableScan Stage: Stage-2 Map Reduce @@ -3645,25 +3311,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-26 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:$INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:$INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} {_col3} {_col4} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col3]] - 1 [Column[value]] - Position of Big Table: 1 - Stage: Stage-16 Map Reduce Alias -> Map Operator Tree: @@ -3706,6 +3353,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:$INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:$INTNAME + TableScan Stage: Stage-1 Map Reduce @@ -3779,26 +3433,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-28 - Map Reduce Local Work - Alias -> Map Local Tables: - join3:join2:join1:bigtbl - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - join3:join2:join1:bigtbl - TableScan - alias: bigtbl - HashTable Sink Operator - condition expressions: - 0 {key1} {key2} {value} - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[key1]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-19 Map Reduce Alias -> Map Operator Tree: @@ -3839,6 +3473,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + join3:join2:join1:bigtbl + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + join3:join2:join1:bigtbl + TableScan + alias: bigtbl Stage: Stage-5 Map Reduce diff --git ql/src/test/results/clientpositive/multiMapJoin2.q.out ql/src/test/results/clientpositive/multiMapJoin2.q.out index 6d0c7d2..034ac51 100644 --- ql/src/test/results/clientpositive/multiMapJoin2.q.out +++ ql/src/test/results/clientpositive/multiMapJoin2.q.out @@ -18,46 +18,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x1) (TOK_TABREF (TOK_TABNAME src1) y1) (= (. (TOK_TABLE_OR_COL x1) key) (. (TOK_TABLE_OR_COL y1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key)))) (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x2) (TOK_TABREF (TOK_TABNAME src1) y2) (= (. (TOK_TABLE_OR_COL x2) key) (. (TOK_TABLE_OR_COL y2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x2) key) key))))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp) key))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL tmp) key))))) STAGE DEPENDENCIES: - Stage-8 is a root stage - Stage-2 depends on stages: Stage-8 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - null-subquery1:tmp-subquery1:y1 - Fetch Operator - limit: -1 - null-subquery2:tmp-subquery2:y2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - null-subquery1:tmp-subquery1:y1 - TableScan - alias: y1 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - null-subquery2:tmp-subquery2:y2 - TableScan - alias: y2 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -133,6 +97,20 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + null-subquery1:tmp-subquery1:y1 + Fetch Operator + limit: -1 + null-subquery2:tmp-subquery2:y2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + null-subquery1:tmp-subquery1:y1 + TableScan + alias: y1 + null-subquery2:tmp-subquery2:y2 + TableScan + alias: y2 Reduce Operator Tree: Extract File Output Operator @@ -271,33 +249,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x1) (TOK_TABREF (TOK_TABNAME src1) y1) (= (. (TOK_TABLE_OR_COL x1) key) (. (TOK_TABLE_OR_COL y1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key)))) (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x2) (TOK_TABREF (TOK_TABNAME src1) y2) (= (. (TOK_TABLE_OR_COL x2) key) (. (TOK_TABLE_OR_COL y2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x2) key) key))))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp) key))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL tmp) key))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-7 depends on stages: Stage-9 - Stage-8 depends on stages: Stage-7 - Stage-2 depends on stages: Stage-8 + Stage-7 is a root stage + Stage-2 depends on stages: Stage-7 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - null-subquery1:tmp-subquery1:y1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - null-subquery1:tmp-subquery1:y1 - TableScan - alias: y1 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-7 Map Reduce Alias -> Map Operator Tree: @@ -330,26 +286,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work - - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - null-subquery2:tmp-subquery2:y2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - null-subquery2:tmp-subquery2:y2 - TableScan - alias: y2 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 + Alias -> Map Local Tables: + null-subquery1:tmp-subquery1:y1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + null-subquery1:tmp-subquery1:y1 + TableScan + alias: y1 Stage: Stage-2 Map Reduce @@ -408,6 +352,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + null-subquery2:tmp-subquery2:y2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + null-subquery2:tmp-subquery2:y2 + TableScan + alias: y2 Reduce Operator Tree: Extract File Output Operator @@ -541,8 +493,7 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-4 is a root stage - Stage-6 depends on stages: Stage-4 - Stage-2 depends on stages: Stage-6 + Stage-2 depends on stages: Stage-4 Stage-0 is a root stage STAGE PLANS: @@ -594,26 +545,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - null-subquery2:tmp-subquery2:y2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - null-subquery2:tmp-subquery2:y2 - TableScan - alias: y2 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -671,6 +602,14 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + null-subquery2:tmp-subquery2:y2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + null-subquery2:tmp-subquery2:y2 + TableScan + alias: y2 Reduce Operator Tree: Extract File Output Operator @@ -798,41 +737,17 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x1) (TOK_TABREF (TOK_TABNAME src1) y1) (= (. (TOK_TABLE_OR_COL x1) key) (. (TOK_TABLE_OR_COL y1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) key)))) tmp1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x2) (TOK_TABREF (TOK_TABNAME src1) y2) (= (. (TOK_TABLE_OR_COL x2) key) (. (TOK_TABLE_OR_COL y2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x2) key) key)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x2) key)))) tmp2) (= (. (TOK_TABLE_OR_COL tmp1) key) (. (TOK_TABLE_OR_COL tmp2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp1) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL tmp1) key)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL cnt))))) STAGE DEPENDENCIES: - Stage-17 is a root stage - Stage-2 depends on stages: Stage-17 - Stage-12 depends on stages: Stage-2, Stage-8 , consists of Stage-15, Stage-16, Stage-3 - Stage-15 has a backup stage: Stage-3 - Stage-10 depends on stages: Stage-15 + Stage-2 is a root stage + Stage-12 depends on stages: Stage-2, Stage-8 , consists of Stage-10, Stage-11, Stage-3 + Stage-10 has a backup stage: Stage-3 Stage-4 depends on stages: Stage-3, Stage-10, Stage-11 Stage-5 depends on stages: Stage-4 - Stage-16 has a backup stage: Stage-3 - Stage-11 depends on stages: Stage-16 + Stage-11 has a backup stage: Stage-3 Stage-3 - Stage-18 is a root stage - Stage-8 depends on stages: Stage-18 + Stage-8 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-17 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp2:y2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp2:y2 - TableScan - alias: y2 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -874,6 +789,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp2:y2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp2:y2 + TableScan + alias: y2 Reduce Operator Tree: Group By Operator bucketGroup: false @@ -898,25 +821,6 @@ STAGE PLANS: Stage: Stage-12 Conditional Operator - Stage: Stage-15 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-10 Map Reduce Alias -> Map Operator Tree: @@ -957,6 +861,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-4 Map Reduce @@ -1028,25 +939,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-16 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-11 Map Reduce Alias -> Map Operator Tree: @@ -1087,6 +979,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-3 Map Reduce @@ -1147,26 +1046,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-18 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp1:y1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp1:y1 - TableScan - alias: y1 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-8 Map Reduce Alias -> Map Operator Tree: @@ -1208,6 +1087,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp1:y1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp1:y1 + TableScan + alias: y1 Reduce Operator Tree: Group By Operator bucketGroup: false @@ -1312,47 +1199,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x1) (TOK_TABREF (TOK_TABNAME src1) y1) (= (. (TOK_TABLE_OR_COL x1) key) (. (TOK_TABLE_OR_COL y1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) key)))) tmp1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x2) (TOK_TABREF (TOK_TABNAME src1) y2) (= (. (TOK_TABLE_OR_COL x2) key) (. (TOK_TABLE_OR_COL y2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x2) key) key)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x2) key)))) tmp2) (= (. (TOK_TABLE_OR_COL tmp1) key) (. (TOK_TABLE_OR_COL tmp2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp1) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL tmp1) key)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL cnt))))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-2 depends on stages: Stage-9 + Stage-2 is a root stage Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp1:y1 - Fetch Operator - limit: -1 - tmp2:y2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp1:y1 - TableScan - alias: y1 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - tmp2:y2 - TableScan - alias: y2 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -1430,6 +1281,20 @@ STAGE PLANS: tag: 1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp1:y1 + Fetch Operator + limit: -1 + tmp2:y2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp1:y1 + TableScan + alias: y1 + tmp2:y2 + TableScan + alias: y2 Reduce Operator Tree: Demux Operator Group By Operator @@ -1651,16 +1516,13 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-7 is a root stage - Stage-10 depends on stages: Stage-2, Stage-7 , consists of Stage-12, Stage-13, Stage-3 - Stage-12 has a backup stage: Stage-3 - Stage-8 depends on stages: Stage-12 + Stage-10 depends on stages: Stage-2, Stage-7 , consists of Stage-8, Stage-9, Stage-3 + Stage-8 has a backup stage: Stage-3 Stage-4 depends on stages: Stage-3, Stage-8, Stage-9 Stage-5 depends on stages: Stage-4 - Stage-13 has a backup stage: Stage-3 - Stage-9 depends on stages: Stage-13 + Stage-9 has a backup stage: Stage-3 Stage-3 - Stage-14 is a root stage - Stage-2 depends on stages: Stage-14 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: @@ -1715,25 +1577,6 @@ STAGE PLANS: Stage: Stage-10 Conditional Operator - Stage: Stage-12 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-8 Map Reduce Alias -> Map Operator Tree: @@ -1774,6 +1617,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-4 Map Reduce @@ -1845,25 +1695,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-13 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - Stage: Stage-9 Map Reduce Alias -> Map Operator Tree: @@ -1904,6 +1735,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-3 Map Reduce @@ -1964,26 +1802,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-14 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp2:y2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp2:y2 - TableScan - alias: y2 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -2025,6 +1843,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp2:y2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp2:y2 + TableScan + alias: y2 Reduce Operator Tree: Group By Operator bucketGroup: false @@ -2129,32 +1955,11 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) key)))) tmp1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x2) (TOK_TABREF (TOK_TABNAME src1) y2) (= (. (TOK_TABLE_OR_COL x2) key) (. (TOK_TABLE_OR_COL y2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x2) key) key)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x2) key)))) tmp2) (= (. (TOK_TABLE_OR_COL tmp1) key) (. (TOK_TABLE_OR_COL tmp2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp1) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL tmp1) key)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL cnt))))) STAGE DEPENDENCIES: - Stage-7 is a root stage - Stage-2 depends on stages: Stage-7 + Stage-2 is a root stage Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - tmp2:y2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - tmp2:y2 - TableScan - alias: y2 - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -2220,6 +2025,14 @@ STAGE PLANS: tag: 1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + tmp2:y2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp2:y2 + TableScan + alias: y2 Reduce Operator Tree: Demux Operator Group By Operator @@ -2447,31 +2260,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME part_table) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) STAGE DEPENDENCIES: - Stage-5 is a root stage - Stage-2 depends on stages: Stage-5 + Stage-2 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-5 - Map Reduce Local Work - Alias -> Map Local Tables: - y - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - y - TableScan - alias: y - HashTable Sink Operator - condition expressions: - 0 - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -2504,6 +2296,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + y + TableScan + alias: y Reduce Operator Tree: Group By Operator aggregations: @@ -2583,35 +2383,12 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_TABREF (TOK_TABNAME src) 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_SELEXPR (. (TOK_TABLE_OR_COL a) key))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL a) key)))) tmp) (TOK_TABREF (TOK_TABNAME src) c) (= (. (TOK_TABLE_OR_COL tmp) key) (. (TOK_TABLE_OR_COL c) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL c) key))))) (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_TABREF (TOK_TABNAME src) 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_SELEXPR (. (TOK_TABLE_OR_COL a) key))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL a) key)))) tmp) (TOK_TABREF (TOK_TABNAME src) c) (= (. (TOK_TABLE_OR_COL tmp) key) (. (TOK_TABLE_OR_COL c) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL c) key)))))) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-17 is a root stage - Stage-2 depends on stages: Stage-17 - Stage-16 depends on stages: Stage-2, Stage-7 - Stage-4 depends on stages: Stage-16 - Stage-18 is a root stage - Stage-7 depends on stages: Stage-18 + Stage-2 is a root stage + Stage-4 depends on stages: Stage-2, Stage-7 + Stage-7 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-17 - Map Reduce Local Work - Alias -> Map Local Tables: - null-subquery2:x-subquery2:tmp:a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - null-subquery2:x-subquery2:tmp:a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-2 Map Reduce Alias -> Map Operator Tree: @@ -2653,6 +2430,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + null-subquery2:x-subquery2:tmp:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + null-subquery2:x-subquery2:tmp:a + TableScan + alias: a Reduce Operator Tree: Group By Operator bucketGroup: false @@ -2674,41 +2459,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - Stage: Stage-16 - Map Reduce Local Work - Alias -> Map Local Tables: - null-subquery1:x-subquery1:c - Fetch Operator - limit: -1 - null-subquery2:x-subquery2:c - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - null-subquery1:x-subquery1:c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - null-subquery2:x-subquery2:c - TableScan - alias: c - HashTable Sink Operator - condition expressions: - 0 - 1 {key} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -2778,26 +2528,20 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work - - Stage: Stage-18 - Map Reduce Local Work - Alias -> Map Local Tables: - null-subquery1:x-subquery1:tmp:b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - null-subquery1:x-subquery1:tmp:b - TableScan - alias: b - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 + Alias -> Map Local Tables: + null-subquery1:x-subquery1:c + Fetch Operator + limit: -1 + null-subquery2:x-subquery2:c + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + null-subquery1:x-subquery1:c + TableScan + alias: c + null-subquery2:x-subquery2:c + TableScan + alias: c Stage: Stage-7 Map Reduce @@ -2840,6 +2584,14 @@ STAGE PLANS: tag: -1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + null-subquery1:x-subquery1:tmp:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + null-subquery1:x-subquery1:tmp:b + TableScan + alias: b Reduce Operator Tree: Group By Operator bucketGroup: false diff --git ql/src/test/results/clientpositive/multi_join_union.q.out ql/src/test/results/clientpositive/multi_join_union.q.out index be6e172..be92318 100644 --- ql/src/test/results/clientpositive/multi_join_union.q.out +++ ql/src/test/results/clientpositive/multi_join_union.q.out @@ -48,9 +48,6 @@ STAGE PLANS: Stage: Stage-8 Map Reduce Local Work Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 c-subquery1:a-subquery1:src13 Fetch Operator limit: -1 @@ -58,18 +55,6 @@ STAGE PLANS: Fetch Operator limit: -1 Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 c-subquery1:a-subquery1:src13 TableScan alias: src13 @@ -179,6 +164,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + c-subquery1:a-subquery1:src13 + c-subquery2:a-subquery2:src14 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + c-subquery1:a-subquery1:src13 + c-subquery2:a-subquery2:src14 Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/reduce_deduplicate_exclude_join.q.out ql/src/test/results/clientpositive/reduce_deduplicate_exclude_join.q.out index fdde8df..2ac20b4 100644 --- ql/src/test/results/clientpositive/reduce_deduplicate_exclude_join.q.out +++ ql/src/test/results/clientpositive/reduce_deduplicate_exclude_join.q.out @@ -7,8 +7,7 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-2 is a root stage - Stage-4 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-4 + Stage-3 depends on stages: Stage-2 Stage-0 is a root stage STAGE PLANS: @@ -49,26 +48,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - 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 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 0 - Stage: Stage-3 Map Reduce Alias -> Map Operator Tree: @@ -107,6 +86,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/select_transform_hint.q.out ql/src/test/results/clientpositive/select_transform_hint.q.out index d9ca1fc..3aafd10 100644 --- ql/src/test/results/clientpositive/select_transform_hint.q.out +++ ql/src/test/results/clientpositive/select_transform_hint.q.out @@ -14,31 +14,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) a) (TOK_TABREF (TOK_TABNAME src) 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 a))) (TOK_SELEXPR (TOK_TRANSFORM (TOK_EXPLIST (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL a) value)) TOK_SERDE TOK_RECORDWRITER 'cat' TOK_SERDE TOK_RECORDREADER (TOK_ALIASLIST tkey tvalue)))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -79,6 +58,14 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/skewjoin.q.out ql/src/test/results/clientpositive/skewjoin.q.out index 2e03b7e..b5b959f 100644 --- ql/src/test/results/clientpositive/skewjoin.q.out +++ ql/src/test/results/clientpositive/skewjoin.q.out @@ -60,9 +60,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-5 depends on stages: Stage-1 , consists of Stage-6, Stage-0 - Stage-6 - Stage-4 depends on stages: Stage-6 + Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-0 + Stage-4 Stage-0 depends on stages: Stage-1, Stage-4 Stage-2 depends on stages: Stage-0 @@ -128,25 +127,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - 1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - 1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {0_VALUE_0} - 1 {1_VALUE_0} - handleSkewJoin: false - keys: - 0 [Column[joinkey0]] - 1 [Column[joinkey0]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -181,6 +161,13 @@ STAGE PLANS: name: default.dest_j1 Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + 1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 1 + TableScan Stage: Stage-0 Move Operator @@ -701,9 +688,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-5 depends on stages: Stage-1 , consists of Stage-6, Stage-2 - Stage-6 - Stage-4 depends on stages: Stage-6 + Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-2 + Stage-4 Stage-2 depends on stages: Stage-1, Stage-4 Stage-0 is a root stage @@ -786,25 +772,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - 1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - 1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {1_VALUE_0} {1_VALUE_1} - handleSkewJoin: false - keys: - 0 [Column[joinkey0]] - 1 [Column[joinkey0]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -845,6 +812,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + 1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 1 + TableScan Stage: Stage-2 Map Reduce @@ -928,9 +902,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-5 depends on stages: Stage-1 , consists of Stage-6, Stage-2 - Stage-6 - Stage-4 depends on stages: Stage-6 + Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-2 + Stage-4 Stage-2 depends on stages: Stage-1, Stage-4 Stage-0 is a root stage @@ -1023,25 +996,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - 1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - 1 - TableScan - HashTable Sink Operator - condition expressions: - 0 - 1 {1_VALUE_0} {1_VALUE_1} - handleSkewJoin: false - keys: - 0 [Column[joinkey0], Column[joinkey1]] - 1 [Column[joinkey0], Column[joinkey1]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -1082,6 +1036,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + 1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 1 + TableScan Stage: Stage-2 Map Reduce @@ -1173,12 +1134,10 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-7 depends on stages: Stage-1 , consists of Stage-8, Stage-9, Stage-2 - Stage-8 - Stage-5 depends on stages: Stage-8 + Stage-7 depends on stages: Stage-1 , consists of Stage-5, Stage-6, Stage-2 + Stage-5 Stage-2 depends on stages: Stage-1, Stage-5, Stage-6 - Stage-9 - Stage-6 depends on stages: Stage-9 + Stage-6 Stage-0 is a root stage STAGE PLANS: @@ -1292,43 +1251,6 @@ STAGE PLANS: Stage: Stage-7 Conditional Operator - Stage: Stage-8 - Map Reduce Local Work - Alias -> Map Local Tables: - 1 - Fetch Operator - limit: -1 - 2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - 1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {0_VALUE_0} - 1 {1_VALUE_0} - 2 - handleSkewJoin: false - keys: - 0 [Column[joinkey0]] - 1 [Column[joinkey0]] - 2 [Column[joinkey0]] - Position of Big Table: 0 - 2 - TableScan - HashTable Sink Operator - condition expressions: - 0 {0_VALUE_0} - 1 {1_VALUE_0} - 2 - handleSkewJoin: false - keys: - 0 [Column[joinkey0]] - 1 [Column[joinkey0]] - 2 [Column[joinkey0]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -1372,6 +1294,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + 1 + Fetch Operator + limit: -1 + 2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 1 + TableScan + 2 + TableScan Stage: Stage-2 Map Reduce @@ -1409,43 +1343,6 @@ STAGE PLANS: output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - 0 - Fetch Operator - limit: -1 - 2 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - 0 - TableScan - HashTable Sink Operator - condition expressions: - 0 {0_VALUE_0} - 1 {1_VALUE_0} - 2 - handleSkewJoin: false - keys: - 0 [Column[joinkey0]] - 1 [Column[joinkey0]] - 2 [Column[joinkey0]] - Position of Big Table: 1 - 2 - TableScan - HashTable Sink Operator - condition expressions: - 0 {0_VALUE_0} - 1 {1_VALUE_0} - 2 - handleSkewJoin: false - keys: - 0 [Column[joinkey0]] - 1 [Column[joinkey0]] - 2 [Column[joinkey0]] - Position of Big Table: 1 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -1489,6 +1386,18 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + 0 + Fetch Operator + limit: -1 + 2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 0 + TableScan + 2 + TableScan Stage: Stage-0 Fetch Operator @@ -1533,31 +1442,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME T1) k) (TOK_TABREF (TOK_TABNAME T1) v) (= (+ (. (TOK_TABLE_OR_COL k) key) 1) (. (TOK_TABLE_OR_COL v) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST v))) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL k) key)))) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL v) val))))))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - v - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - v - TableScan - alias: v - HashTable Sink Operator - condition expressions: - 0 {key} - 1 {val} - handleSkewJoin: false - keys: - 0 [GenericUDFOPPlus(Column[key], Const int 1)] - 1 [GenericUDFBridge(Column[key])] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -1600,6 +1488,14 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + v + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + v + TableScan + alias: v Reduce Operator Tree: Group By Operator aggregations: diff --git ql/src/test/results/clientpositive/skewjoin_noskew.q.out ql/src/test/results/clientpositive/skewjoin_noskew.q.out index fe8da1e..608fac5 100644 --- ql/src/test/results/clientpositive/skewjoin_noskew.q.out +++ ql/src/test/results/clientpositive/skewjoin_noskew.q.out @@ -9,9 +9,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-6 depends on stages: Stage-1 , consists of Stage-7, Stage-2 - Stage-7 - Stage-5 depends on stages: Stage-7 + Stage-6 depends on stages: Stage-1 , consists of Stage-5, Stage-2 + Stage-5 Stage-2 depends on stages: Stage-1, Stage-5 Stage-0 depends on stages: Stage-2 Stage-8 depends on stages: Stage-0 @@ -77,25 +76,6 @@ STAGE PLANS: Stage: Stage-6 Conditional Operator - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - 1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - 1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {0_VALUE_0} {0_VALUE_1} - 1 - handleSkewJoin: false - keys: - 0 [Column[joinkey0]] - 1 [Column[joinkey0]] - Position of Big Table: 0 - Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: @@ -129,6 +109,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + 1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 1 + TableScan Stage: Stage-2 Map Reduce diff --git ql/src/test/results/clientpositive/smb_mapjoin_13.q.out ql/src/test/results/clientpositive/smb_mapjoin_13.q.out index b7dc7f2..ace114f 100644 --- ql/src/test/results/clientpositive/smb_mapjoin_13.q.out +++ ql/src/test/results/clientpositive/smb_mapjoin_13.q.out @@ -257,34 +257,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test_table3) a) (TOK_TABREF (TOK_TABNAME test_table4) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) value)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL a) key))) (TOK_LIMIT 10))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [GenericUDFBridge(Column[key])] - 1 [GenericUDFBridge(Column[value])] - Position of Big Table: 0 - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -340,6 +316,17 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/smb_mapjoin_15.q.out ql/src/test/results/clientpositive/smb_mapjoin_15.q.out index 9adb0d0..f85b6c3 100644 --- ql/src/test/results/clientpositive/smb_mapjoin_15.q.out +++ ql/src/test/results/clientpositive/smb_mapjoin_15.q.out @@ -681,41 +681,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME test_table1) a) (TOK_TABREF (TOK_TABNAME test_table2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) value) (. (TOK_TABLE_OR_COL b) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL a) key))) (TOK_LIMIT 10))) STAGE DEPENDENCIES: - Stage-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 500 dataSize: 7218 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {key2} {value} - 1 {key} {key2} {value} - handleSkewJoin: false - keys: - 0 [Column[key], Column[value]] - 1 [Column[key], Column[value]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {000000_0=[000000_0], 000001_0=[000001_0], 000002_0=[000002_0], 000003_0=[000003_0], 000004_0=[000004_0], 000005_0=[000005_0], 000006_0=[000006_0], 000007_0=[000007_0], 000008_0=[000008_0], 000009_0=[000009_0], 000010_0=[000010_0], 000011_0=[000011_0], 000012_0=[000012_0], 000013_0=[000013_0], 000014_0=[000014_0], 000015_0=[000015_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: @@ -780,6 +749,24 @@ STAGE PLANS: type: string Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 500 dataSize: 7218 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {000000_0=[000000_0], 000001_0=[000001_0], 000002_0=[000002_0], 000003_0=[000003_0], 000004_0=[000004_0], 000005_0=[000005_0], 000006_0=[000006_0], 000007_0=[000007_0], 000008_0=[000008_0], 000009_0=[000009_0], 000010_0=[000010_0], 000011_0=[000011_0], 000012_0=[000012_0], 000013_0=[000013_0], 000014_0=[000014_0], 000015_0=[000015_0]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out index 2011bc3..8b444a8 100644 --- ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out +++ ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out @@ -70,83 +70,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (AND (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL a) part) '1')) (= (. (TOK_TABLE_OR_COL b) part) '1')))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 500 - partition_columns part - rawDataSize 5312 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE - 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 {part=1/000000_0=[part=1/000000_0], part=1/000001_0=[part=1/000001_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: @@ -191,6 +118,66 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 500 + partition_columns part + rawDataSize 5312 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 500 dataSize: 5312 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {part=1/000000_0=[part=1/000000_0], part=1/000001_0=[part=1/000001_0]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out index 3f2f6ab..55d36b2 100644 --- ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out +++ ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out @@ -134,117 +134,10 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_1) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (AND (AND (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL a) part))) (TOK_FUNCTION TOK_ISNOTNULL (. (TOK_TABLE_OR_COL b) part))))) (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-3 is a root stage - Stage-1 depends on stages: Stage-3 + Stage-1 is a root stage Stage-0 is a root stage STAGE PLANS: - Stage: Stage-3 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 1 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 500 - partition_columns part - rawDataSize 5312 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - part 2 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - numFiles 2 - numRows 500 - partition_columns part - rawDataSize 5312 - serialization.ddl struct srcbucket_mapjoin_part_2 { 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part_2 - partition_columns part - serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part_2 - name: default.srcbucket_mapjoin_part_2 - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 1000 dataSize: 10624 basicStatsState: COMPLETE colStatsState: NONE - 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: @@ -288,6 +181,100 @@ STAGE PLANS: type: bigint Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 500 + partition_columns part + rawDataSize 5312 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + part 2 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 500 + partition_columns part + rawDataSize 5312 + serialization.ddl struct srcbucket_mapjoin_part_2 { 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns part + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 1000 dataSize: 10624 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/stats11.q.out ql/src/test/results/clientpositive/stats11.q.out index 44ddc03..8d18093 100644 --- ql/src/test/results/clientpositive/stats11.q.out +++ ql/src/test/results/clientpositive/stats11.q.out @@ -284,8 +284,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-8 depends on stages: Stage-1 , consists of Stage-5, Stage-4, Stage-6 Stage-5 Stage-0 depends on stages: Stage-5, Stage-4, Stage-7 @@ -295,78 +294,6 @@ STAGE DEPENDENCIES: Stage-7 depends on stages: Stage-6 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - numFiles 4 - numRows 0 - partition_columns ds - rawDataSize 0 - serialization.ddl struct srcbucket_mapjoin_part { 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: - bucket_count 4 - bucket_field_name key - columns key,value - columns.types int:string -#### A masked pattern was here #### - name default.srcbucket_mapjoin_part - partition_columns ds - serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcbucket_mapjoin_part - name: default.srcbucket_mapjoin_part - Alias -> Map Local Operator Tree: - b - TableScan - alias: b - Statistics: - numRows: 55 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 0 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - b {srcbucket20.txt=[ds=2008-04-08/srcbucket20.txt, ds=2008-04-08/srcbucket22.txt], srcbucket21.txt=[ds=2008-04-08/srcbucket21.txt, ds=2008-04-08/srcbucket23.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -430,6 +357,66 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + numFiles 4 + numRows 0 + partition_columns ds + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part { 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: + bucket_count 4 + bucket_field_name key + columns key,value + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part + partition_columns ds + serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part + name: default.srcbucket_mapjoin_part + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Statistics: + numRows: 55 dataSize: 5812 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + b {srcbucket20.txt=[ds=2008-04-08/srcbucket20.txt, ds=2008-04-08/srcbucket22.txt], srcbucket21.txt=[ds=2008-04-08/srcbucket21.txt, ds=2008-04-08/srcbucket23.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -825,8 +812,7 @@ ABSTRACT SYNTAX TREE: (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) STAGE DEPENDENCIES: - Stage-9 is a root stage - Stage-1 depends on stages: Stage-9 + Stage-1 is a root stage Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 Stage-4 Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 @@ -836,36 +822,6 @@ STAGE DEPENDENCIES: Stage-6 depends on stages: Stage-5 STAGE PLANS: - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - Statistics: - numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE - GatherStats: false - HashTable Sink Operator - condition expressions: - 0 {key} {value} - 1 {value} - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[key]] - Position of Big Table: 1 - Bucket Mapjoin Context: - Alias Bucket Base File Name Mapping: - a {ds=2008-04-08/srcbucket20.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket21.txt=[srcbucket21.txt], ds=2008-04-08/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket23.txt=[srcbucket21.txt]} - Alias Bucket File Name Mapping: -#### A masked pattern was here #### - Alias Bucket Output File Name Mapping: -#### A masked pattern was here #### - Stage: Stage-1 Map Reduce Alias -> Map Operator Tree: @@ -934,6 +890,24 @@ STAGE PLANS: MultiFileSpray: false Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Statistics: + numRows: 26 dataSize: 2750 basicStatsState: COMPLETE colStatsState: NONE + GatherStats: false + Bucket Mapjoin Context: + Alias Bucket Base File Name Mapping: + a {ds=2008-04-08/srcbucket20.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket21.txt=[srcbucket21.txt], ds=2008-04-08/srcbucket22.txt=[srcbucket20.txt], ds=2008-04-08/srcbucket23.txt=[srcbucket21.txt]} + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### Path -> Alias: #### A masked pattern was here #### Path -> Partition: diff --git ql/src/test/results/clientpositive/subq_where_serialization.q.out ql/src/test/results/clientpositive/subq_where_serialization.q.out index 94840c4..6c4b4ad 100644 --- ql/src/test/results/clientpositive/subq_where_serialization.q.out +++ ql/src/test/results/clientpositive/subq_where_serialization.q.out @@ -7,9 +7,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-5 depends on stages: Stage-1 , consists of Stage-6, Stage-2 - Stage-6 has a backup stage: Stage-2 - Stage-4 depends on stages: Stage-6 + Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-2 + Stage-4 has a backup stage: Stage-2 Stage-2 Stage-0 is a root stage @@ -72,25 +71,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {key} - 1 - handleSkewJoin: false - keys: - 0 [Column[key]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -127,6 +107,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-2 Map Reduce diff --git ql/src/test/results/clientpositive/subquery_in_having.q.out ql/src/test/results/clientpositive/subquery_in_having.q.out index b293ed8..a351f2c 100644 --- ql/src/test/results/clientpositive/subquery_in_having.q.out +++ ql/src/test/results/clientpositive/subquery_in_having.q.out @@ -1108,9 +1108,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-5 is a root stage - Stage-7 depends on stages: Stage-2, Stage-5 , consists of Stage-9, Stage-3 - Stage-9 has a backup stage: Stage-3 - Stage-6 depends on stages: Stage-9 + Stage-7 depends on stages: Stage-2, Stage-5 , consists of Stage-6, Stage-3 + Stage-6 has a backup stage: Stage-3 Stage-3 Stage-10 is a root stage Stage-2 depends on stages: Stage-10 @@ -1186,25 +1185,6 @@ STAGE PLANS: Stage: Stage-7 Conditional Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col2]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: @@ -1244,6 +1224,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan Stage: Stage-3 Map Reduce @@ -1446,9 +1433,8 @@ ABSTRACT SYNTAX TREE: STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-5 depends on stages: Stage-1, Stage-3 , consists of Stage-6, Stage-2 - Stage-6 has a backup stage: Stage-2 - Stage-4 depends on stages: Stage-6 + Stage-5 depends on stages: Stage-1, Stage-3 , consists of Stage-4, Stage-2 + Stage-4 has a backup stage: Stage-2 Stage-2 Stage-3 is a root stage Stage-0 is a root stage @@ -1504,25 +1490,6 @@ STAGE PLANS: Stage: Stage-5 Conditional Operator - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME - TableScan - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} {_col2} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col1]] - 1 [Column[_col0]] - Position of Big Table: 0 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -1562,6 +1529,13 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Local Work: Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan Stage: Stage-2 Map Reduce