Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 45BA617AFF for ; Fri, 17 Oct 2014 20:03:06 +0000 (UTC) Received: (qmail 27106 invoked by uid 500); 17 Oct 2014 20:03:06 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 27065 invoked by uid 500); 17 Oct 2014 20:03:06 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 27054 invoked by uid 99); 17 Oct 2014 20:03:06 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 17 Oct 2014 20:03:06 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 17 Oct 2014 20:02:35 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id A998B2388993; Fri, 17 Oct 2014 20:02:32 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1632659 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/exec/ java/org/apache/hadoop/hive/ql/optimizer/ java/org/apache/hadoop/hive/ql/parse/ java/org/apache/hadoop/hive/ql/plan/ test/results/compiler/plan/ Date: Fri, 17 Oct 2014 20:02:31 -0000 To: commits@hive.apache.org From: gopalv@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20141017200232.A998B2388993@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: gopalv Date: Fri Oct 17 20:02:30 2014 New Revision: 1632659 URL: http://svn.apache.org/r1632659 Log: HIVE-8349: Distinguish between UNIFORM hash-partitioning and AUTOPARALLEL re-partitioning. (Gopal V, reviewed by Gunther Hagleitner) Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDesc.java hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java hive/trunk/ql/src/test/results/compiler/plan/groupby1.q.xml hive/trunk/ql/src/test/results/compiler/plan/groupby2.q.xml hive/trunk/ql/src/test/results/compiler/plan/groupby3.q.xml hive/trunk/ql/src/test/results/compiler/plan/groupby4.q.xml hive/trunk/ql/src/test/results/compiler/plan/groupby5.q.xml hive/trunk/ql/src/test/results/compiler/plan/groupby6.q.xml hive/trunk/ql/src/test/results/compiler/plan/input20.q.xml hive/trunk/ql/src/test/results/compiler/plan/input4.q.xml hive/trunk/ql/src/test/results/compiler/plan/input5.q.xml hive/trunk/ql/src/test/results/compiler/plan/join1.q.xml hive/trunk/ql/src/test/results/compiler/plan/join2.q.xml hive/trunk/ql/src/test/results/compiler/plan/join3.q.xml hive/trunk/ql/src/test/results/compiler/plan/join4.q.xml hive/trunk/ql/src/test/results/compiler/plan/join5.q.xml hive/trunk/ql/src/test/results/compiler/plan/join6.q.xml hive/trunk/ql/src/test/results/compiler/plan/join7.q.xml hive/trunk/ql/src/test/results/compiler/plan/join8.q.xml Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java Fri Oct 17 20:02:30 2014 @@ -55,6 +55,8 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.util.hash.MurmurHash; +import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.UNIFORM; + /** * Reduce Sink Operator sends output to the reduce stage. **/ @@ -110,7 +112,7 @@ public class ReduceSinkOperator extends protected transient int numDistributionKeys; protected transient int numDistinctExprs; protected transient String[] inputAliases; // input aliases of this RS for join (used for PPD) - protected transient boolean autoParallel = false; + protected transient boolean useUniformHash = false; // picks topN K:V pairs from input. protected transient TopNHash reducerHash = new TopNHash(); protected transient HiveKey keyWritable = new HiveKey(); @@ -217,7 +219,7 @@ public class ReduceSinkOperator extends reducerHash.initialize(limit, memUsage, conf.isMapGroupBy(), this); } - autoParallel = conf.isAutoParallel(); + useUniformHash = conf.getReducerTraits().contains(UNIFORM); firstRow = true; initializeChildren(hconf); @@ -339,7 +341,7 @@ public class ReduceSinkOperator extends final int hashCode; // distKeyLength doesn't include tag, but includes buckNum in cachedKeys[0] - if (autoParallel && partitionEval.length > 0) { + if (useUniformHash && partitionEval.length > 0) { hashCode = computeMurmurHash(firstKey); } else { hashCode = computeHashCode(row); Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java Fri Oct 17 20:02:30 2014 @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer; import java.util.ArrayList; +import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -55,6 +56,8 @@ import org.apache.hadoop.hive.ql.plan.Te import org.apache.hadoop.hive.ql.plan.TezWork.VertexType; import org.apache.hadoop.hive.ql.stats.StatsUtils; +import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.FIXED; + public class ReduceSinkMapJoinProc implements NodeProcessor { protected transient Log LOG = LogFactory.getLog(this.getClass().getName()); @@ -167,7 +170,7 @@ public class ReduceSinkMapJoinProc imple if (joinConf.isBucketMapJoin()) { // disable auto parallelism for bucket map joins - parentRS.getConf().setAutoParallel(false); + parentRS.getConf().setReducerTraits(EnumSet.of(FIXED)); numBuckets = (Integer) joinConf.getBigTableBucketNumMapping().values().toArray()[0]; if (joinConf.getCustomBucketMapJoin()) { Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java Fri Oct 17 20:02:30 2014 @@ -18,6 +18,8 @@ package org.apache.hadoop.hive.ql.optimizer; +import java.util.Collection; +import java.util.EnumSet; import java.util.Stack; import org.apache.commons.logging.Log; @@ -31,9 +33,13 @@ import org.apache.hadoop.hive.ql.lib.Nod import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc.ExprNodeDescEqualityWrapper; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; +import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.AUTOPARALLEL; +import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.UNIFORM; + /** * SetReducerParallelism determines how many reducers should * be run for a given reduce sink. @@ -86,7 +92,14 @@ public class SetReducerParallelism imple maxReducers, false); LOG.info("Set parallelism for reduce sink "+sink+" to: "+numReducers); desc.setNumReducers(numReducers); - desc.setAutoParallel(true); + + final Collection keyCols = ExprNodeDescEqualityWrapper.transform(desc.getKeyCols()); + final Collection partCols = ExprNodeDescEqualityWrapper.transform(desc.getPartitionCols()); + if (keyCols != null && keyCols.equals(partCols)) { + desc.setReducerTraits(EnumSet.of(UNIFORM, AUTOPARALLEL)); + } else { + desc.setReducerTraits(EnumSet.of(AUTOPARALLEL)); + } } } else { LOG.info("Number of reducers determined to be: "+desc.getNumReducers()); Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java Fri Oct 17 20:02:30 2014 @@ -57,6 +57,8 @@ import org.apache.hadoop.hive.ql.plan.Un import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; +import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.AUTOPARALLEL; + /** * GenTezUtils is a collection of shared helper methods to produce * TezWork @@ -117,7 +119,7 @@ public class GenTezUtils { reduceWork.setNumReduceTasks(reduceSink.getConf().getNumReducers()); - if (isAutoReduceParallelism && reduceSink.getConf().isAutoParallel()) { + if (isAutoReduceParallelism && reduceSink.getConf().getReducerTraits().contains(AUTOPARALLEL)) { reduceWork.setAutoReduceParallelism(true); // configured limit for reducers Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDesc.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDesc.java?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDesc.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDesc.java Fri Oct 17 20:02:30 2014 @@ -19,6 +19,8 @@ package org.apache.hadoop.hive.ql.plan; import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; import org.apache.hadoop.hive.ql.lib.Node; @@ -125,5 +127,19 @@ public abstract class ExprNodeDesc imple public int hashCode() { return exprNodeDesc == null ? 0 : exprNodeDesc.hashCode(); } + + /* helper function to allow Set()/Collection() operations with ExprNodeDesc */ + public static Collection transform( + Collection descs) { + if (descs == null) { + return null; + } + final Collection wrapped = new ArrayList( + descs.size()); + for (ExprNodeDesc desc : descs) { + wrapped.add(new ExprNodeDescEqualityWrapper(desc)); + } + return wrapped; + } } } Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java Fri Oct 17 20:02:30 2014 @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.plan; import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; import org.apache.commons.logging.Log; @@ -90,7 +91,22 @@ public class ReduceSinkDesc extends Abst //flag used to control how TopN handled for PTF/Windowing partitions. private boolean isPTFReduceSink = false; private boolean skipTag; // Skip writing tags when feeding into mapjoin hashtable - private Boolean autoParallel = null; // Is reducer auto-parallelism enabled, disabled or unset + + public static enum ReducerTraits { + UNSET(0), // unset + FIXED(1), // distribution of keys is fixed + AUTOPARALLEL(2), // can change reducer count (ORDER BY can concat adjacent buckets) + UNIFORM(3); // can redistribute into buckets uniformly (GROUP BY can) + + private final int trait; + + private ReducerTraits(int trait) { + this.trait = trait; + } + }; + + // Is reducer auto-parallelism unset (FIXED, UNIFORM, PARALLEL) + private EnumSet reduceTraits = EnumSet.of(ReducerTraits.UNSET); // Write type, since this needs to calculate buckets differently for updates and deletes private AcidUtils.Operation writeType; @@ -148,7 +164,7 @@ public class ReduceSinkDesc extends Abst desc.setBucketCols(bucketCols); desc.setStatistics(this.getStatistics()); desc.setSkipTag(skipTag); - desc.autoParallel = autoParallel; + desc.reduceTraits = reduceTraits.clone(); return desc; } @@ -361,16 +377,30 @@ public class ReduceSinkDesc extends Abst @Explain(displayName = "auto parallelism", normalExplain = false) public final boolean isAutoParallel() { - return (autoParallel != null) && autoParallel; + return (this.reduceTraits.contains(ReducerTraits.AUTOPARALLEL)); + } + + public final EnumSet getReducerTraits() { + return this.reduceTraits; } - public final void setAutoParallel(final boolean autoParallel) { + public final void setReducerTraits(EnumSet traits) { // we don't allow turning on auto parallel once it has been // explicitly turned off. That is to avoid scenarios where // auto parallelism could break assumptions about number of // reducers or hash function. - if (this.autoParallel == null || this.autoParallel == true) { - this.autoParallel = autoParallel; + + boolean wasUnset = this.reduceTraits.remove(ReducerTraits.UNSET); + + if (this.reduceTraits.contains(ReducerTraits.FIXED)) { + return; + } else if (traits.contains(ReducerTraits.FIXED)) { + this.reduceTraits.removeAll(EnumSet.of( + ReducerTraits.AUTOPARALLEL, + ReducerTraits.UNIFORM)); + this.reduceTraits.addAll(traits); + } else { + this.reduceTraits.addAll(traits); } } Modified: hive/trunk/ql/src/test/results/compiler/plan/groupby1.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/groupby1.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/groupby1.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/groupby1.q.xml Fri Oct 17 20:02:30 2014 @@ -429,6 +429,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/groupby2.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/groupby2.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/groupby2.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/groupby2.q.xml Fri Oct 17 20:02:30 2014 @@ -309,6 +309,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/groupby3.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/groupby3.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/groupby3.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/groupby3.q.xml Fri Oct 17 20:02:30 2014 @@ -364,6 +364,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/groupby4.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/groupby4.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/groupby4.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/groupby4.q.xml Fri Oct 17 20:02:30 2014 @@ -259,6 +259,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/groupby5.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/groupby5.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/groupby5.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/groupby5.q.xml Fri Oct 17 20:02:30 2014 @@ -281,6 +281,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/groupby6.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/groupby6.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/groupby6.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/groupby6.q.xml Fri Oct 17 20:02:30 2014 @@ -259,6 +259,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/input20.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/input20.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/input20.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/input20.q.xml Fri Oct 17 20:02:30 2014 @@ -292,6 +292,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/input4.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/input4.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/input4.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/input4.q.xml Fri Oct 17 20:02:30 2014 @@ -443,6 +443,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/input5.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/input5.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/input5.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/input5.q.xml Fri Oct 17 20:02:30 2014 @@ -448,6 +448,14 @@ + + + + + UNSET + + + -1 Modified: hive/trunk/ql/src/test/results/compiler/plan/join1.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/join1.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/join1.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/join1.q.xml Fri Oct 17 20:02:30 2014 @@ -487,6 +487,14 @@ + + + + + UNSET + + + 1 @@ -890,6 +898,12 @@ + + + + + + Modified: hive/trunk/ql/src/test/results/compiler/plan/join2.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/join2.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/join2.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/join2.q.xml Fri Oct 17 20:02:30 2014 @@ -465,6 +465,14 @@ + + + + + UNSET + + + @@ -741,6 +749,12 @@ + + + + + + 1 @@ -1944,6 +1958,12 @@ + + + + + + 1 @@ -2273,6 +2293,12 @@ + + + + + + Modified: hive/trunk/ql/src/test/results/compiler/plan/join3.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/join3.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/join3.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/join3.q.xml Fri Oct 17 20:02:30 2014 @@ -534,6 +534,14 @@ + + + + + UNSET + + + 1 @@ -920,6 +928,12 @@ + + + + + + 2 @@ -1284,6 +1298,12 @@ + + + + + + Modified: hive/trunk/ql/src/test/results/compiler/plan/join4.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/join4.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/join4.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/join4.q.xml Fri Oct 17 20:02:30 2014 @@ -336,6 +336,14 @@ + + + + + UNSET + + + @@ -911,6 +919,12 @@ + + + + + + 1 Modified: hive/trunk/ql/src/test/results/compiler/plan/join5.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/join5.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/join5.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/join5.q.xml Fri Oct 17 20:02:30 2014 @@ -336,6 +336,14 @@ + + + + + UNSET + + + @@ -911,6 +919,12 @@ + + + + + + 1 Modified: hive/trunk/ql/src/test/results/compiler/plan/join6.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/join6.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/join6.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/join6.q.xml Fri Oct 17 20:02:30 2014 @@ -336,6 +336,14 @@ + + + + + UNSET + + + @@ -911,6 +919,12 @@ + + + + + + 1 Modified: hive/trunk/ql/src/test/results/compiler/plan/join7.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/join7.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/join7.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/join7.q.xml Fri Oct 17 20:02:30 2014 @@ -401,6 +401,14 @@ + + + + + UNSET + + + @@ -976,6 +984,12 @@ + + + + + + 1 @@ -1515,6 +1529,12 @@ + + + + + + 2 Modified: hive/trunk/ql/src/test/results/compiler/plan/join8.q.xml URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/compiler/plan/join8.q.xml?rev=1632659&r1=1632658&r2=1632659&view=diff ============================================================================== --- hive/trunk/ql/src/test/results/compiler/plan/join8.q.xml (original) +++ hive/trunk/ql/src/test/results/compiler/plan/join8.q.xml Fri Oct 17 20:02:30 2014 @@ -336,6 +336,14 @@ + + + + + UNSET + + + @@ -952,6 +960,12 @@ + + + + + + 1