diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java index c66cf0c..1a4c4ef 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java @@ -17,18 +17,12 @@ */ package org.apache.hadoop.hive.ql.exec; -import java.util.Iterator; -import java.util.List; import java.util.Map; import org.apache.commons.logging.Log; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.io.IOContext; -import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.util.ReflectionUtils; public class ExecMapperContext { @@ -110,26 +104,6 @@ public class ExecMapperContext { this.lastInputFile = lastInputFile; } - - private void setUpFetchOpContext(FetchOperator fetchOp, String alias) - throws Exception { - String currentInputFile = HiveConf.getVar(jc, - HiveConf.ConfVars.HADOOPMAPFILENAME); - BucketMapJoinContext bucketMatcherCxt = this.localWork - .getBucketMapjoinContext(); - Class bucketMatcherCls = bucketMatcherCxt - .getBucketMatcherClass(); - BucketMatcher bucketMatcher = (BucketMatcher) ReflectionUtils.newInstance( - bucketMatcherCls, null); - bucketMatcher.setAliasBucketFileNameMapping(bucketMatcherCxt - .getAliasBucketFileNameMapping()); - - List aliasFiles = bucketMatcher.getAliasBucketFiles(currentInputFile, - bucketMatcherCxt.getMapJoinBigTableAlias(), alias); - Iterator iter = aliasFiles.iterator(); - fetchOp.setupContext(iter, null); - } - public String getCurrentInputFile() { currentInputFile = this.ioCxt.getInputFile(); return currentInputFile; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java index f19a5c0..d1992c7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java @@ -143,6 +143,10 @@ public class FetchOperator implements Serializable { } else { isNativeTable = true; } + setupExecContext(); + } + + private void setupExecContext() { if (hasVC || work.getSplitSample() != null) { context = new ExecMapperContext(); if (operator != null) { @@ -536,6 +540,7 @@ public class FetchOperator implements Serializable { context.clear(); context = null; } + this.currTbl = null; this.currPath = null; this.iterPath = null; this.iterPartDesc = null; @@ -546,21 +551,16 @@ public class FetchOperator implements Serializable { } /** - * used for bucket map join. there is a hack for getting partitionDesc. bucket map join right now - * only allow one partition present in bucket map join. + * used for bucket map join */ - public void setupContext(Iterator iterPath, Iterator iterPartDesc) { - this.iterPath = iterPath; - this.iterPartDesc = iterPartDesc; - if (iterPartDesc == null) { - if (work.isNotPartitioned()) { - this.currTbl = work.getTblDesc(); - } else { - // hack, get the first. - List listParts = work.getPartDesc(); - currPart = listParts.isEmpty() ? null : listParts.get(0); - } + public void setupContext(List paths) { + this.iterPath = paths.iterator(); + if (work.isNotPartitioned()) { + this.currTbl = work.getTblDesc(); + } else { + this.iterPartDesc = work.getPartDescs(paths).iterator(); } + setupExecContext(); } /** diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java index f646df3..21462a0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java @@ -28,7 +28,6 @@ import java.util.ArrayList; import java.util.Calendar; import java.util.Collection; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; @@ -432,8 +431,7 @@ public class MapredLocalTask extends Task implements Serializab List aliasFiles = bucketMatcher.getAliasBucketFiles(currentInputFile, bucketMatcherCxt .getMapJoinBigTableAlias(), alias); - Iterator iter = aliasFiles.iterator(); - fetchOp.setupContext(iter, null); + fetchOp.setupContext(aliasFiles); } @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java index 5fe38de..5ac159f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java @@ -17,10 +17,11 @@ */ package org.apache.hadoop.hive.ql.exec; +import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; @@ -37,12 +38,14 @@ import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.ql.util.ObjectPair; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.PriorityQueue; import org.apache.hadoop.util.ReflectionUtils; /** @@ -57,7 +60,8 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp .getName()); private MapredLocalWork localWork = null; - private Map fetchOperators; + private Map mergeQueues; + transient ArrayList[] keyWritables; transient ArrayList[] nextKeyWritables; RowContainer>[] nextGroupStorage; @@ -141,42 +145,51 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp } localWorkInited = true; this.localWork = localWork; - fetchOperators = new HashMap(); + mergeQueues = new HashMap(); - Map fetchOpJobConfMap = new HashMap(); // create map local operators - for (Map.Entry entry : localWork.getAliasToFetchWork() - .entrySet()) { - JobConf jobClone = new JobConf(hconf); - Operator tableScan = localWork.getAliasToWork() - .get(entry.getKey()); - if(tableScan instanceof TableScanOperator) { - ArrayList list = ((TableScanOperator)tableScan).getNeededColumnIDs(); - if (list != null) { - ColumnProjectionUtils.appendReadColumnIDs(jobClone, list); - } - } else { - ColumnProjectionUtils.setFullyReadColumns(jobClone); + Map aliasToFetchWork = localWork.getAliasToFetchWork(); + Map> aliasToWork = localWork.getAliasToWork(); + + for (Map.Entry entry : aliasToFetchWork.entrySet()) { + String alias = entry.getKey(); + FetchWork fetchWork = entry.getValue(); + + Operator forwardOp = aliasToWork.get(alias); + forwardOp.setExecContext(getExecContext()); + + JobConf jobClone = cloneJobConf(hconf, forwardOp); + FetchOperator fetchOp = new FetchOperator(fetchWork, jobClone); + forwardOp.initialize(jobClone, new ObjectInspector[]{fetchOp.getOutputObjectInspector()}); + fetchOp.clearFetchContext(); + + MergeQueue mergeQueue = new MergeQueue(alias, fetchWork, jobClone); + + mergeQueues.put(alias, mergeQueue); + l4j.info("fetchoperator for " + alias + " initialized"); + } + } + + private JobConf cloneJobConf(Configuration hconf, Operator op) { + JobConf jobClone = new JobConf(hconf); + if (op instanceof TableScanOperator) { + List list = ((TableScanOperator)op).getNeededColumnIDs(); + if (list != null) { + ColumnProjectionUtils.appendReadColumnIDs(jobClone, list); } - FetchOperator fetchOp = new FetchOperator(entry.getValue(),jobClone); - fetchOpJobConfMap.put(fetchOp, jobClone); - fetchOperators.put(entry.getKey(), fetchOp); - l4j.info("fetchoperator for " + entry.getKey() + " created"); + } else { + ColumnProjectionUtils.setFullyReadColumns(jobClone); } + return jobClone; + } - for (Map.Entry entry : fetchOperators.entrySet()) { - Operator forwardOp = localWork.getAliasToWork() - .get(entry.getKey()); - // All the operators need to be initialized before process - forwardOp.setExecContext(this.getExecContext()); - FetchOperator fetchOp = entry.getValue(); - JobConf jobConf = fetchOpJobConfMap.get(fetchOp); - if (jobConf == null) { - jobConf = this.getExecContext().getJc(); + private byte tagForAlias(String alias) { + for (Map.Entry entry : tagToAlias.entrySet()) { + if (entry.getValue().equals(alias)) { + return entry.getKey(); } - forwardOp.initialize(jobConf, new ObjectInspector[] {fetchOp.getOutputObjectInspector()}); - l4j.info("fetchoperator for " + entry.getKey() + " initialized"); } + return -1; } // The input file has changed - load the correct hash bucket @@ -195,9 +208,9 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp joinFinalLeftData(); } // set up the fetch operator for the new input file. - for (Map.Entry entry : fetchOperators.entrySet()) { + for (Map.Entry entry : mergeQueues.entrySet()) { String alias = entry.getKey(); - FetchOperator fetchOp = entry.getValue(); + MergeQueue fetchOp = entry.getValue(); fetchOp.clearFetchContext(); setUpFetchOpContext(fetchOp, alias); } @@ -217,7 +230,6 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp } byte alias = (byte) tag; - // compute keys and values as StandardObjects // compute keys and values as StandardObjects ArrayList key = JoinUtil.computeKeys(row, joinKeys.get(alias), @@ -402,7 +414,7 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp private int compareKeys (ArrayList k1, ArrayList k2) { int ret = 0; - // join keys have difference sizes? + // join keys have difference sizes? ret = k1.size() - k2.size(); if (ret != 0) { return ret; @@ -474,34 +486,32 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp } } - private void setUpFetchOpContext(FetchOperator fetchOp, String alias) { + private void setUpFetchOpContext(MergeQueue fetchOp, String alias) throws HiveException { String currentInputFile = getExecContext().getCurrentInputFile(); - BucketMapJoinContext bucketMatcherCxt = localWork.getBucketMapjoinContext(); - Class bucketMatcherCls = bucketMatcherCxt - .getBucketMatcherClass(); - BucketMatcher bucketMatcher = (BucketMatcher) ReflectionUtils.newInstance( - bucketMatcherCls, null); + BucketMapJoinContext bucketMatcherCxt = localWork.getBucketMapjoinContext(); + Class bucketMatcherCls = bucketMatcherCxt.getBucketMatcherClass(); + BucketMatcher bucketMatcher = ReflectionUtils.newInstance(bucketMatcherCls, null); getExecContext().setFileId(bucketMatcherCxt.createFileId(currentInputFile)); LOG.info("set task id: " + getExecContext().getFileId()); bucketMatcher.setAliasBucketFileNameMapping(bucketMatcherCxt .getAliasBucketFileNameMapping()); + List aliasFiles = bucketMatcher.getAliasBucketFiles(currentInputFile, bucketMatcherCxt.getMapJoinBigTableAlias(), alias); - Iterator iter = aliasFiles.iterator(); - fetchOp.setupContext(iter, null); + fetchOp.setupContext(aliasFiles); } private void fetchOneRow(byte tag) { - if (fetchOperators != null) { - String tble = this.tagToAlias.get(tag); - FetchOperator fetchOp = fetchOperators.get(tble); + if (mergeQueues != null) { + String table = tagToAlias.get(tag); + MergeQueue fetchOp = mergeQueues.get(table); Operator forwardOp = localWork.getAliasToWork() - .get(tble); + .get(table); try { InspectableObject row = fetchOp.getNextRow(); if (row == null) { @@ -535,9 +545,9 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp if (inputFileChanged || !firstFetchHappened) { //set up the fetch operator for the new input file. - for (Map.Entry entry : fetchOperators.entrySet()) { + for (Map.Entry entry : mergeQueues.entrySet()) { String alias = entry.getKey(); - FetchOperator fetchOp = entry.getValue(); + MergeQueue fetchOp = entry.getValue(); fetchOp.clearFetchContext(); setUpFetchOpContext(fetchOp, alias); } @@ -563,8 +573,8 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp localWorkInited = false; super.closeOp(abort); - if (fetchOperators != null) { - for (Map.Entry entry : fetchOperators.entrySet()) { + if (mergeQueues != null) { + for (Map.Entry entry : mergeQueues.entrySet()) { Operator forwardOp = localWork .getAliasToWork().get(entry.getKey()); forwardOp.close(abort); @@ -591,4 +601,141 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator imp public OperatorType getType() { return OperatorType.MAPJOIN; } + + // uses hadoop primary queue + // element (Integer) is index to segments (FetchOperator[]) + private class MergeQueue extends PriorityQueue { + + String alias; + FetchWork fetchWork; + JobConf jobConf; + + transient int counter; + + transient FetchOperator[] segments; + transient List keyFields; + transient List keyFieldOIs; + + transient Pair[] keys; + transient Integer minSegment; + + public MergeQueue(String alias, FetchWork fetchWork, JobConf jobConf) { + this.alias = alias; + this.fetchWork = fetchWork; + this.jobConf = jobConf; + } + + // paths = files from small alias, initializes a FetchOperator for each file + // size of paths can be vary in further develop (bucket per partition, etc.) + public void setupContext(List paths) throws HiveException { + int segmentLen = paths.size(); + FetchOperator[] segments = segmentsForSize(segmentLen); + for (int i = 0 ; i < segmentLen; i++) { + Path path = paths.get(i); + if (segments[i] == null) { + segments[i] = new FetchOperator(fetchWork, new JobConf(jobConf)); + } + segments[i].setupContext(Arrays.asList(path)); + } + initialize(segmentLen); + for (int i = 0; i < segmentLen; i++) { + if (nextHive(i)) { + put(i); + } + } + counter = 0; + } + + private FetchOperator[] segmentsForSize(int segmentLen) { + if (segments == null || segments.length < segmentLen) { + FetchOperator[] newSegments = new FetchOperator[segmentLen]; + Pair[] newKeys = new Pair[segmentLen]; + if (segments != null) { + System.arraycopy(segments, 0, newSegments, 0, segments.length); + System.arraycopy(keys, 0, newKeys, 0, keys.length); + } + segments = newSegments; + keys = newKeys; + } + return segments; + } + + public void clearFetchContext() throws HiveException { + if (segments != null) { + for (FetchOperator op : segments) { + if (op != null) { + op.clearFetchContext(); + } + } + } + } + + protected boolean lessThan(Object a, Object b) { + return compareKeys(keys[(Integer) a].getFirst(), keys[(Integer)b].getFirst()) < 0; + } + + public final InspectableObject getNextRow() throws IOException { + if (minSegment != null) { + adjustPriorityQueue(minSegment); + } + Integer current = top(); + if (current == null) { + LOG.info("MergeQueue forwarded " + counter + " rows"); + return null; + } + counter++; + return keys[minSegment = current].getSecond(); + } + + private void adjustPriorityQueue(Integer current) throws IOException { + if (nextIO(current)) { + adjustTop(); // sort + } else { + pop(); + } + } + + // wraping for exception handling + private boolean nextHive(Integer current) throws HiveException { + try { + return next(current); + } catch (IOException e) { + throw new HiveException(e); + } + } + + // wraping for exception handling + private boolean nextIO(Integer current) throws IOException { + try { + return next(current); + } catch (HiveException e) { + throw new IOException(e); + } + } + + // return true if current segment(FetchOperator) has next row + private boolean next(Integer current) throws IOException, HiveException { + if (keyFields == null) { + // joinKeys/joinKeysOI are initialized after making merge queue, so setup lazily at runtime + byte tag = tagForAlias(alias); + keyFields = joinKeys.get(tag); + keyFieldOIs = joinKeysObjectInspectors.get(tag); + } + InspectableObject nextRow = segments[current].getNextRow(); + if (nextRow != null) { + if (keys[current] == null) { + keys[current] = new Pair(); + } + // todo this should be changed to be evaluated lazily, especially for single segment case + keys[current].setFirst(JoinUtil.computeKeys(nextRow.o, keyFields, keyFieldOIs)); + keys[current].setSecond(nextRow); + return true; + } + keys[current] = null; + return false; + } + } + + private class Pair extends ObjectPair, InspectableObject> { + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java index 89741b6..56cfdb6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java @@ -242,15 +242,6 @@ public class SortedMergeBucketMapJoinOptimizer implements Transform { if (tso == null) { return false; } - if (pos != op.getConf().getPosBigTable()) { - // currently, a file from a big table can be joined with only 1 file from a small table - for (List files : - op.getConf().getAliasBucketFileNameMapping().get(alias).values()) { - if (files != null && files.size() > 1) { - return false; - } - } - } List keys = op.getConf().getKeys().get((byte) pos); // get all join columns from join keys stored in MapJoinDesc diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java index 313e772..65c39d6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java @@ -193,6 +193,17 @@ public class FetchWork implements Serializable { } /** + * @return the partDescs for paths + */ + public List getPartDescs(List paths) { + List parts = new ArrayList(paths.size()); + for (Path path : paths) { + parts.add(partDesc.get(partDir.indexOf(path.getParent().toString()))); + } + return parts; + } + + /** * @param partDesc * the partDesc to set */ diff --git ql/src/java/org/apache/hadoop/hive/ql/util/ObjectPair.java ql/src/java/org/apache/hadoop/hive/ql/util/ObjectPair.java new file mode 100644 index 0000000..185bcb8 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/util/ObjectPair.java @@ -0,0 +1,47 @@ +/** + * 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.util; + +public class ObjectPair { + private F first; + private S second; + + public ObjectPair() {} + + public ObjectPair(F first, S second) { + this.first = first; + this.second = second; + } + + public F getFirst() { + return first; + } + + public void setFirst(F first) { + this.first = first; + } + + public S getSecond() { + return second; + } + + public void setSecond(S second) { + this.second = second; + } +} diff --git ql/src/test/queries/clientpositive/bucketcontext_1.q ql/src/test/queries/clientpositive/bucketcontext_1.q index 9bfa123..5b01399 100644 --- ql/src/test/queries/clientpositive/bucketcontext_1.q +++ ql/src/test/queries/clientpositive/bucketcontext_1.q @@ -19,6 +19,5 @@ explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN buc select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; set hive.optimize.bucketmapjoin.sortedmerge = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git ql/src/test/queries/clientpositive/bucketcontext_2.q ql/src/test/queries/clientpositive/bucketcontext_2.q index ca1bd8c..f952f2e 100644 --- ql/src/test/queries/clientpositive/bucketcontext_2.q +++ ql/src/test/queries/clientpositive/bucketcontext_2.q @@ -17,6 +17,5 @@ explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN buc select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; set hive.optimize.bucketmapjoin.sortedmerge = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; - +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git ql/src/test/queries/clientpositive/bucketcontext_3.q ql/src/test/queries/clientpositive/bucketcontext_3.q index 1915384..461fbb1 100644 --- ql/src/test/queries/clientpositive/bucketcontext_3.q +++ ql/src/test/queries/clientpositive/bucketcontext_3.q @@ -17,6 +17,5 @@ explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN buc select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; set hive.optimize.bucketmapjoin.sortedmerge = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; - +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git ql/src/test/queries/clientpositive/bucketcontext_4.q ql/src/test/queries/clientpositive/bucketcontext_4.q index f864b41..366da44 100644 --- ql/src/test/queries/clientpositive/bucketcontext_4.q +++ ql/src/test/queries/clientpositive/bucketcontext_4.q @@ -19,7 +19,5 @@ explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN buc select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; set hive.optimize.bucketmapjoin.sortedmerge = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; - - +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git ql/src/test/queries/clientpositive/bucketcontext_5.q ql/src/test/queries/clientpositive/bucketcontext_5.q new file mode 100644 index 0000000..411fed3 --- /dev/null +++ ql/src/test/queries/clientpositive/bucketcontext_5.q @@ -0,0 +1,18 @@ +-- small no part, 4 bucket & big no part, 2 bucket +CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; +load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; +load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; +load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; + +CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; +load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git ql/src/test/queries/clientpositive/bucketcontext_6.q ql/src/test/queries/clientpositive/bucketcontext_6.q new file mode 100644 index 0000000..204d1e9 --- /dev/null +++ ql/src/test/queries/clientpositive/bucketcontext_6.q @@ -0,0 +1,21 @@ +-- small no part, 4 bucket & big 2 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; +load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; +load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; +load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; + +CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); + +load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git ql/src/test/results/clientpositive/bucketcontext_2.q.out ql/src/test/results/clientpositive/bucketcontext_2.q.out index a0bdf6b..abf91ca 100644 --- ql/src/test/results/clientpositive/bucketcontext_2.q.out +++ ql/src/test/results/clientpositive/bucketcontext_2.q.out @@ -337,40 +337,11 @@ 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-4 is a root stage - Stage-1 depends on stages: Stage-4 + Stage-1 is a root stage Stage-2 depends on stages: Stage-1 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - 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 - 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: @@ -378,7 +349,7 @@ STAGE PLANS: TableScan alias: b GatherStats: false - Map Join Operator + Sorted Merge Bucket Map Join Operator condition map: Inner Join 0 to 1 condition expressions: @@ -404,8 +375,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Local Work: - Map Reduce Local Work Needs Tagging: false Path -> Alias: #### A masked pattern was here #### @@ -584,3 +553,16 @@ STAGE PLANS: limit: -1 +PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +928 diff --git ql/src/test/results/clientpositive/bucketcontext_3.q.out ql/src/test/results/clientpositive/bucketcontext_3.q.out index 5a89119..41d2586 100644 --- ql/src/test/results/clientpositive/bucketcontext_3.q.out +++ ql/src/test/results/clientpositive/bucketcontext_3.q.out @@ -287,40 +287,11 @@ 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-4 is a root stage - Stage-1 depends on stages: Stage-4 + Stage-1 is a root stage Stage-2 depends on stages: Stage-1 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - 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 - 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: @@ -328,7 +299,7 @@ STAGE PLANS: TableScan alias: b GatherStats: false - Map Join Operator + Sorted Merge Bucket Map Join Operator condition map: Inner Join 0 to 1 condition expressions: @@ -354,8 +325,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Local Work: - Map Reduce Local Work Needs Tagging: false Path -> Alias: #### A masked pattern was here #### @@ -484,3 +453,16 @@ STAGE PLANS: limit: -1 +PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +928 diff --git ql/src/test/results/clientpositive/bucketcontext_4.q.out ql/src/test/results/clientpositive/bucketcontext_4.q.out index abf6bc2..ea3ead5 100644 --- ql/src/test/results/clientpositive/bucketcontext_4.q.out +++ ql/src/test/results/clientpositive/bucketcontext_4.q.out @@ -299,40 +299,11 @@ 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-4 is a root stage - Stage-1 depends on stages: Stage-4 + Stage-1 is a root stage Stage-2 depends on stages: Stage-1 Stage-0 is a root stage STAGE PLANS: - Stage: Stage-4 - Map Reduce Local Work - Alias -> Map Local Tables: - 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 - 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: @@ -340,7 +311,7 @@ STAGE PLANS: TableScan alias: b GatherStats: false - Map Join Operator + Sorted Merge Bucket Map Join Operator condition map: Inner Join 0 to 1 condition expressions: @@ -366,8 +337,6 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false - Local Work: - Map Reduce Local Work Needs Tagging: false Path -> Alias: #### A masked pattern was here #### @@ -496,3 +465,16 @@ STAGE PLANS: limit: -1 +PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +928 diff --git ql/src/test/results/clientpositive/bucketcontext_5.q.out ql/src/test/results/clientpositive/bucketcontext_5.q.out new file mode 100644 index 0000000..2348aea --- /dev/null +++ ql/src/test/results/clientpositive/bucketcontext_5.q.out @@ -0,0 +1,441 @@ +PREHOOK: query: -- small no part, 4 bucket & big no part, 2 bucket +CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- small no part, 4 bucket & big no part, 2 bucket +CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_small +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_big +PREHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +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-4 is a root stage + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-4 + Map Reduce Local Work + Alias -> Map Local Tables: + 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 + 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: + b + TableScan + alias: b + GatherStats: false + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + Local Work: + Map Reduce Local Work + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: bucket_big + 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 + numFiles 2 + numPartitions 0 + numRows 0 + 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 + numFiles 2 + numPartitions 0 + numRows 0 + 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 + name: default.bucket_big + name: default.bucket_big + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Select Operator + Select Operator + Group By Operator + aggregations: + expr: count() + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: -mr-10002 + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: bigint + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_small +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_small +#### A masked pattern was here #### +464 +PREHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +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-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + GatherStats: false + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: bucket_big + 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 + numFiles 2 + numPartitions 0 + numRows 0 + 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 + numFiles 2 + numPartitions 0 + numRows 0 + 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 + name: default.bucket_big + name: default.bucket_big + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Select Operator + Select Operator + Group By Operator + aggregations: + expr: count() + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: -mr-10002 + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: bigint + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_small +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_small +#### A masked pattern was here #### +464 diff --git ql/src/test/results/clientpositive/bucketcontext_6.q.out ql/src/test/results/clientpositive/bucketcontext_6.q.out new file mode 100644 index 0000000..c0bae1c --- /dev/null +++ ql/src/test/results/clientpositive/bucketcontext_6.q.out @@ -0,0 +1,567 @@ +PREHOOK: query: -- small no part, 4 bucket & big 2 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- small no part, 4 bucket & big 2 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_small +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +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-4 is a root stage + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-4 + Map Reduce Local Work + Alias -> Map Local Tables: + 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 + 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: + b + TableScan + alias: b + GatherStats: false + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + Local Work: + Map Reduce Local Work + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=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: + 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 + numFiles 2 + numPartitions 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 + numFiles 4 + numPartitions 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 5500 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + 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: + 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 + numFiles 2 + numPartitions 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 + numFiles 4 + numPartitions 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 5500 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Select Operator + Select Operator + Group By Operator + aggregations: + expr: count() + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: -mr-10002 + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: bigint + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +#### A masked pattern was here #### +928 +PREHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +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-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + GatherStats: false + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=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: + 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 + numFiles 2 + numPartitions 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 + numFiles 4 + numPartitions 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 5500 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + 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: + 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 + numFiles 2 + numPartitions 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 + numFiles 4 + numPartitions 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 5500 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Select Operator + Select Operator + Group By Operator + aggregations: + expr: count() + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + Needs Tagging: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: -mr-10002 + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns + columns.types + escape.delim \ + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: bigint + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +#### A masked pattern was here #### +928 diff --git serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java index 927c084..23180cf 100644 --- serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java +++ serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.serde2; import java.util.ArrayList; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.StringUtils; @@ -36,7 +37,7 @@ public final class ColumnProjectionUtils { * is included in the list, RCFile's reader will not skip its value. * */ - public static void setReadColumnIDs(Configuration conf, ArrayList ids) { + public static void setReadColumnIDs(Configuration conf, List ids) { String id = toReadColumnIDString(ids); setReadColumnIDConf(conf, id); } @@ -46,8 +47,7 @@ public final class ColumnProjectionUtils { * is included in the list, RCFile's reader will not skip its value. * */ - public static void appendReadColumnIDs(Configuration conf, - ArrayList ids) { + public static void appendReadColumnIDs(Configuration conf, List ids) { String id = toReadColumnIDString(ids); if (id != null) { String old = conf.get(READ_COLUMN_IDS_CONF_STR, null); @@ -69,7 +69,7 @@ public final class ColumnProjectionUtils { conf.set(READ_COLUMN_IDS_CONF_STR, id); } - private static String toReadColumnIDString(ArrayList ids) { + private static String toReadColumnIDString(List ids) { String id = null; if (ids != null) { for (int i = 0; i < ids.size(); i++) {