You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2009/06/04 03:21:35 UTC

svn commit: r781633 [4/13] - in /hadoop/hive/trunk: ./ common/src/java/org/apache/hadoop/hive/conf/ data/scripts/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ...

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/fetchWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/fetchWork.java?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/fetchWork.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/fetchWork.java Thu Jun  4 01:21:30 2009
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
@@ -28,10 +29,10 @@
 public class fetchWork implements Serializable {
   private static final long serialVersionUID = 1L;
 
-  private Path tblDir;
+  private String    tblDir;
   private tableDesc tblDesc;
 
-  private List<Path> partDir;
+  private List<String>        partDir;
   private List<partitionDesc> partDesc;
 
   private int limit;
@@ -43,13 +44,21 @@
 
   public fetchWork() { }
 
-	public fetchWork(Path tblDir, tableDesc tblDesc, int limit) {
+  public fetchWork(String tblDir, tableDesc tblDesc) {
+    this(tblDir, tblDesc, -1);
+  }
+
+	public fetchWork(String tblDir, tableDesc tblDesc, int limit) {
 		this.tblDir = tblDir;
 		this.tblDesc = tblDesc;
 		this.limit = limit;
 	}
 
-	public fetchWork(List<Path> partDir, List<partitionDesc> partDesc, int limit) {
+	public fetchWork(List<String> partDir, List<partitionDesc> partDesc) {
+	  this(partDir, partDesc, -1);
+	}
+	 
+	public fetchWork(List<String> partDir, List<partitionDesc> partDesc, int limit) {
 		this.partDir = partDir;
 		this.partDesc = partDesc;
 		this.limit = limit;
@@ -66,14 +75,21 @@
 	/**
 	 * @return the tblDir
 	 */
-	public Path getTblDir() {
+	public String getTblDir() {
 		return tblDir;
 	}
 
+	 /**
+   * @return the tblDir
+   */
+  public Path getTblDirPath() {
+    return new Path(tblDir);
+  }
+
 	/**
 	 * @param tblDir the tblDir to set
 	 */
-	public void setTblDir(Path tblDir) {
+	public void setTblDir(String tblDir) {
 		this.tblDir = tblDir;
 	}
 
@@ -94,14 +110,41 @@
 	/**
 	 * @return the partDir
 	 */
-	public List<Path> getPartDir() {
+	public List<String> getPartDir() {
 		return partDir;
 	}
 
+
+	public List<Path> getPartDirPath() {
+	  return fetchWork.convertStringToPathArray(partDir);
+	}
+	
+	public static List<String> convertPathToStringArray(List<Path> paths) {
+	   if (paths == null)
+	      return null;
+	    
+	   List<String> pathsStr = new ArrayList<String>();
+	   for (Path path : paths)
+	     pathsStr.add(path.toString());
+	    
+	   return pathsStr;
+	}
+	
+	 public static List<Path> convertStringToPathArray(List<String> paths) {
+     if (paths == null)
+        return null;
+      
+     List<Path> pathsStr = new ArrayList<Path>();
+     for (String path : paths)
+       pathsStr.add(new Path(path));
+      
+     return pathsStr;
+  }
+
 	/**
 	 * @param partDir the partDir to set
 	 */
-	public void setPartDir(List<Path> partDir) {
+	public void setPartDir(List<String> partDir) {
 		this.partDir = partDir;
 	}
 
@@ -133,4 +176,18 @@
 	public void setLimit(int limit) {
 		this.limit = limit;
 	}
+	
+	public String toString() {
+    if (tblDir != null)
+	    return new String ("table = " + tblDir);
+	  
+	  if (partDir == null) 
+	    return "null fetchwork";
+	  	  
+	  String ret = new String("partition = ");
+    for (String part : partDir)
+     ret = ret.concat(part);
+	  
+    return ret;
+  }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/joinDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/joinDesc.java?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/joinDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/joinDesc.java Thu Jun  4 01:21:30 2009
@@ -38,35 +38,35 @@
   public static final int RIGHT_OUTER_JOIN = 2;
   public static final int FULL_OUTER_JOIN = 3;
 
-  // alias to columns mapping
-  private Map<Byte, ArrayList<exprNodeDesc>> exprs;
+  // alias to key mapping
+  private Map<Byte, List<exprNodeDesc>> exprs;
   
   // No outer join involved
-  private boolean noOuterJoin;
-
-  private joinCond[] conds;
+  protected boolean noOuterJoin;
 
+  protected joinCond[] conds;
+  
   public joinDesc() { }
   
-  public joinDesc(final Map<Byte, ArrayList<exprNodeDesc>> exprs, final boolean noOuterJoin, final joinCond[] conds) {
+  public joinDesc(final Map<Byte, List<exprNodeDesc>> exprs, final boolean noOuterJoin, final joinCond[] conds) {
     this.exprs = exprs;
     this.noOuterJoin = noOuterJoin;
     this.conds = conds;
   }
   
-  public joinDesc(final Map<Byte, ArrayList<exprNodeDesc>> exprs) {
+  public joinDesc(final Map<Byte, List<exprNodeDesc>> exprs) {
     this.exprs = exprs;
     this.noOuterJoin = true;
     this.conds = null;
   }
 
-  public joinDesc(final Map<Byte, ArrayList<exprNodeDesc>> exprs, final joinCond[] conds) {
+  public joinDesc(final Map<Byte, List<exprNodeDesc>> exprs, final joinCond[] conds) {
     this.exprs = exprs;
     this.noOuterJoin = false;
     this.conds = conds;
   }
   
-  public Map<Byte, ArrayList<exprNodeDesc>> getExprs() {
+  public Map<Byte, List<exprNodeDesc>> getExprs() {
     return this.exprs;
   }
 
@@ -78,7 +78,7 @@
     
     LinkedHashMap<Byte, String> ret = new LinkedHashMap<Byte, String>();
     
-    for(Map.Entry<Byte, ArrayList<exprNodeDesc>> ent: getExprs().entrySet()) {
+    for(Map.Entry<Byte, List<exprNodeDesc>> ent: getExprs().entrySet()) {
       StringBuilder sb = new StringBuilder();
       boolean first = true;
       if (ent.getValue() != null) {
@@ -99,7 +99,7 @@
     return ret;
   }
   
-  public void setExprs(final Map<Byte, ArrayList<exprNodeDesc>> exprs) {
+  public void setExprs(final Map<Byte, List<exprNodeDesc>> exprs) {
     this.exprs = exprs;
   }
 

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapJoinDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapJoinDesc.java?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapJoinDesc.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapJoinDesc.java Thu Jun  4 01:21:30 2009
@@ -0,0 +1,113 @@
+/**
+ * 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.plan;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Map Join operator Descriptor implementation.
+ * 
+ */
+@explain(displayName="Common Join Operator")
+public class mapJoinDesc extends joinDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private Map<Byte, List<exprNodeDesc>> keys;
+  private tableDesc keyTblDesc;
+  private List<tableDesc> valueTblDescs;
+  
+  private int posBigTable;
+  
+  public mapJoinDesc() { }
+
+  public mapJoinDesc(final Map<Byte, List<exprNodeDesc>> keys, 
+                     final tableDesc keyTblDesc, 
+                     final Map<Byte, List<exprNodeDesc>> values,
+                     final List<tableDesc> valueTblDescs,
+                     final int posBigTable,
+                     final joinCond[] conds) {
+    super(values, conds);
+    this.keys        = keys;
+    this.keyTblDesc  = keyTblDesc;
+    this.valueTblDescs = valueTblDescs;
+    this.posBigTable = posBigTable;
+  }
+
+  /**
+   * @return the keys
+   */
+  @explain(displayName="keys")
+  public Map<Byte, List<exprNodeDesc>> getKeys() {
+    return keys;
+  }
+
+  /**
+   * @param keys the keys to set
+   */
+  public void setKeys(Map<Byte, List<exprNodeDesc>> keys) {
+    this.keys = keys;
+  }
+
+  /**
+   * @return the position of the big table not in memory
+   */
+  @explain(displayName="Position of Big Table")
+  public int getPosBigTable() {
+    return posBigTable;
+  }
+
+  /**
+   * @param posBigTable the position of the big table not in memory
+   */
+  public void setPosBigTable(int posBigTable) {
+    this.posBigTable = posBigTable;
+  }
+
+  /**
+   * @return the keyTblDesc
+   */
+  public tableDesc getKeyTblDesc() {
+    return keyTblDesc;
+  }
+
+  /**
+   * @param keyTblDesc the keyTblDesc to set
+   */
+  public void setKeyTblDesc(tableDesc keyTblDesc) {
+    this.keyTblDesc = keyTblDesc;
+  }
+
+  /**
+   * @return the valueTblDescs
+   */
+  public List<tableDesc> getValueTblDescs() {
+    return valueTblDescs;
+  }
+
+  /**
+   * @param valueTblDescs the valueTblDescs to set
+   */
+  public void setValueTblDescs(List<tableDesc> valueTblDescs) {
+    this.valueTblDescs = valueTblDescs;
+  }
+}

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredLocalWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredLocalWork.java?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredLocalWork.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredLocalWork.java Thu Jun  4 01:21:30 2009
@@ -0,0 +1,64 @@
+/**
+ * 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.plan;
+
+import java.util.*;
+import java.io.*;
+
+import org.apache.hadoop.hive.ql.exec.Operator;
+
+@explain(displayName="Map Reduce Local Work")
+public class mapredLocalWork implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private LinkedHashMap<String, Operator<? extends Serializable>> aliasToWork;
+  private LinkedHashMap<String, fetchWork> aliasToFetchWork;
+
+  public mapredLocalWork() { }
+
+  public mapredLocalWork(final LinkedHashMap<String, Operator<? extends Serializable>> aliasToWork,
+                         final LinkedHashMap<String, fetchWork> aliasToFetchWork) {
+    this.aliasToWork = aliasToWork;
+    this.aliasToFetchWork = aliasToFetchWork;
+  }
+
+  @explain(displayName="Alias -> Map Local Operator Tree")
+  public LinkedHashMap<String, Operator<? extends Serializable>> getAliasToWork() {
+    return aliasToWork;
+  }
+
+  public void setAliasToWork(final LinkedHashMap<String, Operator<? extends Serializable>> aliasToWork) {
+    this.aliasToWork = aliasToWork;
+  }
+
+  /**
+   * @return the aliasToFetchWork
+   */
+  @explain(displayName="Alias -> Map Local Tables")
+  public LinkedHashMap<String, fetchWork> getAliasToFetchWork() {
+    return aliasToFetchWork;
+  }
+
+  /**
+   * @param aliasToFetchWork the aliasToFetchWork to set
+   */
+  public void setAliasToFetchWork(final LinkedHashMap<String, fetchWork> aliasToFetchWork) {
+    this.aliasToFetchWork = aliasToFetchWork;
+  }
+}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredWork.java?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredWork.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredWork.java Thu Jun  4 01:21:30 2009
@@ -36,6 +36,8 @@
   private LinkedHashMap<String,partitionDesc> pathToPartitionInfo;
   
   private LinkedHashMap<String,Operator<? extends Serializable>> aliasToWork;
+  
+  private LinkedHashMap<String, partitionDesc> aliasToPartnInfo;
 
   // map<->reduce interface
   // schema of the map-reduce 'key' object - this is homogeneous
@@ -49,8 +51,12 @@
   private Integer numReduceTasks;
   
   private boolean needsTagging;
+  private mapredLocalWork mapLocalWork;
+
+  public mapredWork() { 
+    this.aliasToPartnInfo = new LinkedHashMap<String, partitionDesc>();
+  }
 
-  public mapredWork() { }
   public mapredWork(
     final String command,
     final LinkedHashMap<String,ArrayList<String>> pathToAliases,
@@ -59,16 +65,20 @@
     final tableDesc keyDesc,
     List<tableDesc> tagToValueDesc,
     final Operator<?> reducer,
-    final Integer numReduceTasks) {
-    this.command = command;
-    this.pathToAliases = pathToAliases;
-    this.pathToPartitionInfo = pathToPartitionInfo;
-    this.aliasToWork = aliasToWork;
-    this.keyDesc = keyDesc;
-    this.tagToValueDesc = tagToValueDesc;
-    this.reducer = reducer;
-    this.numReduceTasks = numReduceTasks;
+    final Integer numReduceTasks,
+    final mapredLocalWork mapLocalWork) {
+      this.command = command;
+      this.pathToAliases = pathToAliases;
+      this.pathToPartitionInfo = pathToPartitionInfo;
+      this.aliasToWork = aliasToWork;
+      this.keyDesc = keyDesc;
+      this.tagToValueDesc = tagToValueDesc;
+      this.reducer = reducer;
+      this.numReduceTasks = numReduceTasks;
+      this.mapLocalWork = mapLocalWork;
+      this.aliasToPartnInfo = new LinkedHashMap<String, partitionDesc>();
   }
+
   public String getCommand() {
     return this.command;
   }
@@ -92,6 +102,21 @@
     this.pathToPartitionInfo = pathToPartitionInfo;
   }
   
+  /**
+   * @return the aliasToPartnInfo
+   */
+  public LinkedHashMap<String, partitionDesc> getAliasToPartnInfo() {
+    return aliasToPartnInfo;
+  }
+  
+  /**
+   * @param aliasToPartnInfo the aliasToPartnInfo to set
+   */
+  public void setAliasToPartnInfo(
+      LinkedHashMap<String, partitionDesc> aliasToPartnInfo) {
+    this.aliasToPartnInfo = aliasToPartnInfo;
+  }
+  
   @explain(displayName="Alias -> Map Operator Tree")
   public LinkedHashMap<String, Operator<? extends Serializable>> getAliasToWork() {
     return this.aliasToWork;
@@ -99,6 +124,23 @@
   public void setAliasToWork(final LinkedHashMap<String,Operator<? extends Serializable>> aliasToWork) {
     this.aliasToWork=aliasToWork;
   }
+
+
+  /**
+   * @return the mapredLocalWork
+   */
+  @explain(displayName="Local Work")
+  public mapredLocalWork getMapLocalWork() {
+    return mapLocalWork;
+  }
+
+  /**
+   * @param mapredLocalWork the mapredLocalWork to set
+   */
+  public void setMapLocalWork(final mapredLocalWork mapLocalWork) {
+    this.mapLocalWork = mapLocalWork;
+  }
+
   public tableDesc getKeyDesc() {
     return this.keyDesc;
   }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/selectDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/selectDesc.java?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/selectDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/selectDesc.java Thu Jun  4 01:21:30 2009
@@ -25,7 +25,13 @@
   private static final long serialVersionUID = 1L;
   private java.util.ArrayList<org.apache.hadoop.hive.ql.plan.exprNodeDesc> colList;
   private boolean selectStar;
+  private boolean selStarNoCompute;
   public selectDesc() { }
+
+  public selectDesc(final boolean selStarNoCompute) {
+    this.selStarNoCompute = selStarNoCompute;
+  }
+
   public selectDesc(final java.util.ArrayList<org.apache.hadoop.hive.ql.plan.exprNodeDesc> colList) {
     this(colList, false);
   }
@@ -35,6 +41,14 @@
     this.colList = colList;
     this.selectStar = selectStar;
   }
+
+  public selectDesc(
+    final java.util.ArrayList<org.apache.hadoop.hive.ql.plan.exprNodeDesc> colList, final boolean selectStar,
+    final boolean selStarNoCompute) {
+    this.colList = colList;
+    this.selectStar = selectStar;
+    this.selStarNoCompute = selStarNoCompute;
+  }
   
   @explain(displayName="expressions")
   public java.util.ArrayList<org.apache.hadoop.hive.ql.plan.exprNodeDesc> getColList() {
@@ -56,4 +70,18 @@
   public void setSelectStar(boolean selectStar) {
     this.selectStar = selectStar;
   }
+
+  /**
+   * @return the selStarNoCompute
+   */
+  public boolean isSelStarNoCompute() {
+    return selStarNoCompute;
+  }
+
+  /**
+   * @param selStarNoCompute the selStarNoCompute to set
+   */
+  public void setSelStarNoCompute(boolean selStarNoCompute) {
+    this.selStarNoCompute = selStarNoCompute;
+  }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/util/jdbm/RecordManagerFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/util/jdbm/RecordManagerFactory.java?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/util/jdbm/RecordManagerFactory.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/util/jdbm/RecordManagerFactory.java Thu Jun  4 01:21:30 2009
@@ -117,7 +117,7 @@
         RecordManagerProvider  factory;
 
         provider = options.getProperty( RecordManagerOptions.PROVIDER_FACTORY,
-                                        "jdbm.recman.Provider" );
+                                        "org.apache.hadoop.hive.ql.util.jdbm.recman.Provider" );
 
         try {
             clazz = Class.forName( provider );

Modified: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java (original)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java Thu Jun  4 01:21:30 2009
@@ -89,7 +89,7 @@
       op.setConf(filterCtx);
 
       // runtime initialization
-      op.initialize(null, null, null);
+      op.initialize(null, null, new ObjectInspector[]{r[0].oi});
 
       for(InspectableObject oner: r) {
         op.process(oner.o, oner.oi, 0);
@@ -139,7 +139,7 @@
                                            Utilities.defaultTd, false);
       Operator<fileSinkDesc> flop = OperatorFactory.getAndMakeChild(fsd, op);
       
-      op.initialize(new JobConf(TestOperators.class), Reporter.NULL, null);
+      op.initialize(new JobConf(TestOperators.class), Reporter.NULL, new ObjectInspector[]{r[0].oi});
 
       // evaluate on row
       for(int i=0; i<5; i++) {
@@ -185,7 +185,7 @@
       collectDesc cd = new collectDesc (Integer.valueOf(10));
       CollectOperator cdop = (CollectOperator) OperatorFactory.getAndMakeChild(cd, sop);
 
-      op.initialize(new JobConf(TestOperators.class), null, null);
+      op.initialize(new JobConf(TestOperators.class), null, new ObjectInspector[]{r[0].oi});
 
       // evaluate on row
       for(int i=0; i<5; i++) {

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/join2.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/join2.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/join2.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/join2.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,5 @@
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value
+FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key);
+
+
+

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join25.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join25.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join25.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join25.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,19 @@
+set hive.mapjoin.numrows = 2;
+
+drop table dest_j1;
+
+CREATE TABLE dest_j1(key INT, value STRING, val2 STRING) STORED AS TEXTFILE;
+
+EXPLAIN
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value
+FROM src1 x JOIN src y ON (x.key = y.key);
+
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value
+FROM src1 x JOIN src y ON (x.key = y.key);
+
+select * from dest_j1 x order by x.key;
+
+drop table dest_j1;
+

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join26.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join26.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join26.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join26.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,17 @@
+CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE;
+
+EXPLAIN EXTENDED
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value
+FROM src1 x JOIN src y ON (x.key = y.key) 
+JOIN srcpart z ON (x.key = z.key and z.ds='2008-04-08' and z.hr=11);
+
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value
+FROM src1 x JOIN src y ON (x.key = y.key) 
+JOIN srcpart z ON (x.key = z.key and z.ds='2008-04-08' and z.hr=11);
+
+select * from dest_j1 x order by x.key;
+
+drop table dest_j1;
+

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join27.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join27.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join27.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join27.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,17 @@
+drop table dest_j1;
+
+CREATE TABLE dest_j1(key INT, value STRING, val2 STRING) STORED AS TEXTFILE;
+
+EXPLAIN
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value
+FROM src1 x JOIN src y ON (x.value = y.value);
+
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value
+FROM src1 x JOIN src y ON (x.value = y.value);
+
+select * from dest_j1 x order by x.key, x.value;
+
+drop table dest_j1;
+

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join28.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join28.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join28.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join28.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,23 @@
+drop table dest_j1;
+
+CREATE TABLE dest_j1(key STRING, value STRING) STORED AS TEXTFILE;
+
+EXPLAIN
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(z) */ subq.key1, z.value
+FROM
+(SELECT /*+ MAPJOIN(x) */ x.key as key1, x.value as value1, y.key as key2, y.value as value2 
+ FROM src1 x JOIN src y ON (x.key = y.key)) subq
+ JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11);
+
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(z) */ subq.key1, z.value
+FROM
+(SELECT /*+ MAPJOIN(x) */ x.key as key1, x.value as value1, y.key as key2, y.value as value2 
+ FROM src1 x JOIN src y ON (x.key = y.key)) subq
+ JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11);
+
+select * from dest_j1 x order by x.key;
+
+drop table dest_j1;
+

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join29.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join29.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join29.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join29.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,18 @@
+drop TABLE dest_j1;
+
+CREATE TABLE dest_j1(key STRING, cnt1 INT, cnt2 INT);
+
+EXPLAIN 
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(subq1) */ subq1.key, subq1.cnt, subq2.cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN 
+     (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key);
+
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(subq1) */ subq1.key, subq1.cnt, subq2.cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN 
+     (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key);
+
+select * from dest_j1 x order by x.key;
+
+drop TABLE dest_j1;

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join30.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join30.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join30.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join30.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,14 @@
+drop TABLE dest_j1;
+
+CREATE TABLE dest_j1(key INT, cnt INT);
+
+EXPLAIN
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(x) */ x.key, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key;
+
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(x) */ x.key, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key;
+
+select * from dest_j1 x order by x.key;
+
+drop TABLE dest_j1;

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join31.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join31.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join31.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join31.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,20 @@
+drop TABLE dest_j1;
+
+CREATE TABLE dest_j1(key STRING, cnt INT);
+
+EXPLAIN 
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(subq1) */ subq1.key, count(1) as cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN 
+     (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key)
+group by subq1.key;
+
+INSERT OVERWRITE TABLE dest_j1 
+SELECT /*+ MAPJOIN(subq1) */ subq1.key, count(1) as cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN 
+     (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key)
+group by subq1.key;
+
+select * from dest_j1 x order by x.key;
+
+drop TABLE dest_j1;

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join32.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join32.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join32.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join32.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,17 @@
+CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE;
+
+EXPLAIN EXTENDED
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x,z) */ x.key, z.value, y.value
+FROM src1 x JOIN src y ON (x.key = y.key) 
+JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11);
+
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x,z) */ x.key, z.value, y.value
+FROM src1 x JOIN src y ON (x.key = y.key) 
+JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11);
+
+select * from dest_j1 x order by x.key;
+
+drop table dest_j1;
+

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join33.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join33.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join33.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join33.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,17 @@
+CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE;
+
+EXPLAIN EXTENDED
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x) */ x.key, z.value, y.value
+FROM src1 x JOIN src y ON (x.key = y.key) 
+JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11);
+
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x) */ x.key, z.value, y.value
+FROM src1 x JOIN src y ON (x.key = y.key) 
+JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11);
+
+select * from dest_j1 x order by x.key;
+
+drop table dest_j1;
+

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join34.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join34.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join34.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join34.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,27 @@
+drop table dest_j1;
+
+CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE;
+
+EXPLAIN EXTENDED
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, subq1.value
+FROM 
+( SELECT x.key as key, x.value as value from src x where x.key < 20
+     UNION ALL
+  SELECT x1.key as key, x1.value as value from src x1 where x1.key > 100
+) subq1
+JOIN src1 x ON (x.key = subq1.key);
+
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, subq1.value
+FROM 
+( SELECT x.key as key, x.value as value from src x where x.key < 20
+     UNION ALL
+  SELECT x1.key as key, x1.value as value from src x1 where x1.key > 100
+) subq1
+JOIN src1 x ON (x.key = subq1.key);
+
+select * from dest_j1 x order by x.key;
+
+drop table dest_j1;
+

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/join35.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/join35.q?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/join35.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/join35.q Thu Jun  4 01:21:30 2009
@@ -0,0 +1,27 @@
+drop table dest_j1;
+
+CREATE TABLE dest_j1(key STRING, value STRING, val2 INT) STORED AS TEXTFILE;
+
+EXPLAIN EXTENDED
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, subq1.cnt
+FROM 
+( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key
+     UNION ALL
+  SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key
+) subq1
+JOIN src1 x ON (x.key = subq1.key);
+
+INSERT OVERWRITE TABLE dest_j1
+SELECT /*+ MAPJOIN(x) */ x.key, x.value, subq1.cnt
+FROM 
+( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key
+     UNION ALL
+  SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key
+) subq1
+JOIN src1 x ON (x.key = subq1.key);
+
+select * from dest_j1 x order by x.key;
+
+drop table dest_j1;
+

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/join2.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/join2.q.out?rev=781633&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/join2.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/join2.q.out Thu Jun  4 01:21:30 2009
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: Map Join cannot be performed with Outer join

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out Thu Jun  4 01:21:30 2009
@@ -52,7 +52,7 @@
 
 query: SELECT * FROM SRC x where x.key = 10 CLUSTER BY x.key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/415464870/184820352.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/214678633/80263680.10000
 10	val_10
 query: EXPLAIN
 SELECT * FROM SRC x  where x.key = 20 CLUSTER BY key
@@ -108,7 +108,7 @@
 
 query: SELECT * FROM SRC x where x.key = 20 CLUSTER BY key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/483996565/25535146.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/216341669/198536245.10000
 20	val_20
 query: EXPLAIN
 SELECT x.* FROM SRC x where x.key = 20 CLUSTER BY key
@@ -164,7 +164,7 @@
 
 query: SELECT x.* FROM SRC x where x.key = 20 CLUSTER BY key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/67500406/610168339.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/323533340/5794695.10000
 20	val_20
 query: EXPLAIN
 SELECT x.*  FROM SRC x where x.key = 20 CLUSTER BY x.key
@@ -220,7 +220,7 @@
 
 query: SELECT x.*  FROM SRC x where x.key = 20 CLUSTER BY x.key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/1062209591/78302640.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1217098935/331281713.10000
 20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY key
@@ -276,7 +276,7 @@
 
 query: SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/14719539/136351528.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/31096986/775877790.10000
 20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY x.key
@@ -332,7 +332,7 @@
 
 query: SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY x.key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/168361199/328930285.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1311539054/408350350.10000
 20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1  FROM SRC x where x.key = 20 CLUSTER BY v1
@@ -388,7 +388,7 @@
 
 query: SELECT x.key, x.value as v1  FROM SRC x where x.key = 20 CLUSTER BY v1
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/218669889/821061905.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/410641125/337645701.10000
 20	val_20
 query: EXPLAIN
 SELECT y.* from (SELECT x.* FROM SRC x CLUSTER BY x.key) y where y.key = 20
@@ -450,7 +450,7 @@
 
 query: SELECT y.* from (SELECT x.* FROM SRC x CLUSTER BY x.key) y where y.key = 20
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/1411175272/114925452.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/20608200/132489807.10000
 20	val_20
 query: EXPLAIN 
 SELECT x.key, x.value as v1, y.key  FROM SRC x JOIN SRC y ON (x.key = y.key)  where x.key = 20 CLUSTER BY v1
@@ -528,23 +528,23 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/213518946/254574448.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 1
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 1
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 0
-                  type: string
-                  expr: 1
-                  type: string
-                  expr: 2
-                  type: string
+        /data/users/njain/hive2/hive2/build/ql/tmp/180519365/1223155.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 1
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 1
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 0
+                    type: string
+                    expr: 1
+                    type: string
+                    expr: 2
+                    type: string
       Reduce Operator Tree:
         Extract
           File Output Operator
@@ -561,7 +561,7 @@
 
 query: SELECT x.key, x.value as v1, y.key  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/115523540/275557180.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/304705152/1520883527.10000
 20	val_20	20
 query: EXPLAIN 
 SELECT x.key, x.value as v1, y.*  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1
@@ -639,25 +639,25 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/1145795613/25764450.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 1
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 1
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 0
-                  type: string
-                  expr: 1
-                  type: string
-                  expr: 2
-                  type: string
-                  expr: 3
-                  type: string
+        /data/users/njain/hive2/hive2/build/ql/tmp/273445284/1259872115.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 1
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 1
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 0
+                    type: string
+                    expr: 1
+                    type: string
+                    expr: 2
+                    type: string
+                    expr: 3
+                    type: string
       Reduce Operator Tree:
         Extract
           File Output Operator
@@ -674,7 +674,7 @@
 
 query: SELECT x.key, x.value as v1, y.*  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/1597231071/54379941.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/224961014/306327097.10000
 20	val_20	20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1, y.*  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY x.key
@@ -752,25 +752,25 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/186086163/372716927.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 0
-                  type: string
-                  expr: 1
-                  type: string
-                  expr: 2
-                  type: string
-                  expr: 3
-                  type: string
+        /data/users/njain/hive2/hive2/build/ql/tmp/389423102/202869602.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 0
+                    type: string
+                    expr: 1
+                    type: string
+                    expr: 2
+                    type: string
+                    expr: 3
+                    type: string
       Reduce Operator Tree:
         Extract
           File Output Operator
@@ -787,7 +787,7 @@
 
 query: SELECT x.key, x.value as v1, y.*  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY x.key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/895988628/1557073804.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/171221765/388816243.10000
 20	val_20	20	val_20
 query: EXPLAIN
 SELECT x.key, x.value as v1, y.key as yk  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY key
@@ -865,23 +865,23 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/173489852/1494704235.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 0
-                  type: string
-                  expr: 1
-                  type: string
-                  expr: 2
-                  type: string
+        /data/users/njain/hive2/hive2/build/ql/tmp/399836187/1625046097.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 0
+                    type: string
+                    expr: 1
+                    type: string
+                    expr: 2
+                    type: string
       Reduce Operator Tree:
         Extract
           File Output Operator
@@ -898,7 +898,7 @@
 
 query: SELECT x.key, x.value as v1, y.key as yk  FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/144828791/434522525.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/667900999/93186085.10000
 20	val_20	20
 query: EXPLAIN
 SELECT unioninput.*
@@ -1005,7 +1005,7 @@
 ) unioninput
 CLUSTER BY unioninput.key
 Input: default/src
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/247260620/164962735.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/730558416/61341521.10000
 0	val_0
 0	val_0
 0	val_0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1.q.out Thu Jun  4 01:21:30 2009
@@ -45,19 +45,19 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/121598851/279205958.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/3613518/365997197.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -103,7 +103,7 @@
 Output: default/dest_g1
 query: SELECT dest_g1.* FROM dest_g1
 Input: default/dest_g1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/71965672/1050923207.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/233907446/231615542.10000
 0	0.0
 10	10.0
 100	200.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_limit.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_limit.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_limit.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_limit.q.out Thu Jun  4 01:21:30 2009
@@ -59,15 +59,15 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/3130862/578287351.10002 
-          Reduce Output Operator
-            sort order: 
-            tag: -1
-            value expressions:
-                  expr: 0
-                  type: string
-                  expr: 1
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/77797555/24296274.10002 
+            Reduce Output Operator
+              sort order: 
+              tag: -1
+              value expressions:
+                    expr: 0
+                    type: string
+                    expr: 1
+                    type: double
       Reduce Operator Tree:
         Extract
           Limit
@@ -102,7 +102,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/212729573/95177349.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1165028443/472212754.10000
 0	0.0
 10	10.0
 100	200.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_map_skew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_map_skew.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_map_skew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_map_skew.q.out Thu Jun  4 01:21:30 2009
@@ -52,19 +52,19 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/571062527/279369936.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/1694296539/661835976.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -110,7 +110,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/94711228/451211575.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/24366777/439991944.10000
 0	0.0
 10	10.0
 100	200.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out Thu Jun  4 01:21:30 2009
@@ -48,21 +48,21 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/35355459/37601301.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: bigint
-                  expr: 2
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/485798928/730742911.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: bigint
+                    expr: 2
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -114,7 +114,7 @@
 Output: default/dest_g2
 query: SELECT dest_g2.* FROM dest_g2
 Input: default/dest_g2
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/214974209/1305159333.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/5024073/19494424.10000
 0	1	00.0
 1	71	116414.0
 2	69	225571.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out Thu Jun  4 01:21:30 2009
@@ -63,21 +63,21 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/580419027/35834303.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: bigint
-                  expr: 2
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/197258813/105701362.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: bigint
+                    expr: 2
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -129,7 +129,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/541951294/114930476.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1713198569/54590123.10000
 0	1	00.0
 1	71	116414.0
 2	69	225571.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3.q.out Thu Jun  4 01:21:30 2009
@@ -48,21 +48,21 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/736681081/169179727.10002 
-          Reduce Output Operator
-            sort order: 
-            tag: -1
-            value expressions:
-                  expr: 0
-                  type: double
-                  expr: 1
-                  type: string
-                  expr: 2
-                  type: string
-                  expr: 3
-                  type: string
-                  expr: 4
-                  type: string
+        /data/users/njain/hive2/hive2/build/ql/tmp/270940600/835157322.10002 
+            Reduce Output Operator
+              sort order: 
+              tag: -1
+              value expressions:
+                    expr: 0
+                    type: double
+                    expr: 1
+                    type: string
+                    expr: 2
+                    type: string
+                    expr: 3
+                    type: string
+                    expr: 4
+                    type: string
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -122,5 +122,5 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/47601886/767510798.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/167235948/1313312877.10000
 130091.0	260.182	256.10355987055016	98.0	0.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3_map_skew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3_map_skew.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3_map_skew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3_map_skew.q.out Thu Jun  4 01:21:30 2009
@@ -70,21 +70,21 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/97967328/397511110.10002 
-          Reduce Output Operator
-            sort order: 
-            tag: -1
-            value expressions:
-                  expr: 0
-                  type: double
-                  expr: 1
-                  type: string
-                  expr: 2
-                  type: string
-                  expr: 3
-                  type: string
-                  expr: 4
-                  type: string
+        /data/users/njain/hive2/hive2/build/ql/tmp/250385923/119575383.10002 
+            Reduce Output Operator
+              sort order: 
+              tag: -1
+              value expressions:
+                    expr: 0
+                    type: double
+                    expr: 1
+                    type: string
+                    expr: 2
+                    type: string
+                    expr: 3
+                    type: string
+                    expr: 4
+                    type: string
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -144,5 +144,5 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/200485216/757320674.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/483439487/1053812433.10000
 130091.0	260.182	256.10355987055016	98.0	0.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby4.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby4.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby4.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby4.q.out Thu Jun  4 01:21:30 2009
@@ -45,16 +45,16 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/155743537/194052245.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
+        /data/users/njain/hive2/hive2/build/ql/tmp/427345693/640920280.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
       Reduce Operator Tree:
         Group By Operator
           keys:
@@ -91,7 +91,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/88502906/569100809.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/19444424/1257782123.10000
 0
 1
 2

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby5.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby5.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby5.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby5.q.out Thu Jun  4 01:21:30 2009
@@ -48,19 +48,19 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/496157129/694276302.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/184711117/547495960.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -109,7 +109,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/898371644/800050208.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/102051313/48306509.10000
 0	0.0
 10	10.0
 100	200.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6.q.out Thu Jun  4 01:21:30 2009
@@ -45,16 +45,16 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/605013417/275946039.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
+        /data/users/njain/hive2/hive2/build/ql/tmp/378777975/56851564.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
       Reduce Operator Tree:
         Group By Operator
           keys:
@@ -91,7 +91,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/26109249/3810410.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/690607981/10565214.10000
 0
 1
 2

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6_map_skew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6_map_skew.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6_map_skew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6_map_skew.q.out Thu Jun  4 01:21:30 2009
@@ -50,16 +50,16 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/19945520/12512881.10002 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
+        /data/users/njain/hive2/hive2/build/ql/tmp/341443584/940777218.10002 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
       Reduce Operator Tree:
         Group By Operator
           keys:
@@ -96,7 +96,7 @@
 Output: default/dest1
 query: SELECT dest1.* FROM dest1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/2643090/1744448740.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1048827/342876636.10000
 0
 1
 2

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map.q.out Thu Jun  4 01:21:30 2009
@@ -99,19 +99,19 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/148712980/235409749.10004 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/267196387/57369739.10004 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -150,7 +150,7 @@
 Output: default/dest2
 query: SELECT DEST1.* FROM DEST1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/46974594/51174778.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1043668036/332712287.10000
 0	0.0
 10	10.0
 100	200.0
@@ -462,7 +462,7 @@
 98	196.0
 query: SELECT DEST2.* FROM DEST2
 Input: default/dest2
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/294954367/121849485.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/473793334/1186860826.10000
 0	0.0
 10	10.0
 100	200.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map_skew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map_skew.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map_skew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map_skew.q.out Thu Jun  4 01:21:30 2009
@@ -72,19 +72,19 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/1783138161/343279396.10004 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/517555/87986579.10004 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -133,19 +133,19 @@
   Stage: Stage-3
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/1783138161/343279396.10005 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: rand()
-                  type: double
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/517555/87986579.10005 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: rand()
+                    type: double
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -165,19 +165,19 @@
   Stage: Stage-4
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/1783138161/343279396.10006 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: double
+        /data/users/njain/hive2/hive2/build/ql/tmp/517555/87986579.10006 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: double
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -216,7 +216,7 @@
 Output: default/dest2
 query: SELECT DEST1.* FROM DEST1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/304069265/587781436.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1583075121/447851227.10000
 0	0.0
 10	10.0
 100	200.0
@@ -528,7 +528,7 @@
 98	196.0
 query: SELECT DEST2.* FROM DEST2
 Input: default/dest2
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/1656610083/39812005.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1089426067/5659897.10000
 0	0.0
 10	10.0
 100	200.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_noskew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_noskew.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_noskew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_noskew.q.out Thu Jun  4 01:21:30 2009
@@ -85,19 +85,19 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/1287144918/310129488.10004 
-          Reduce Output Operator
-            key expressions:
-                  expr: key
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: key
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: substr(value, 5)
-                  type: string
+        /data/users/njain/hive2/hive2/build/ql/tmp/81820008/280821152.10004 
+            Reduce Output Operator
+              key expressions:
+                    expr: key
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: key
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: substr(value, 5)
+                    type: string
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -136,7 +136,7 @@
 Output: default/dest2
 query: SELECT DEST1.* FROM DEST1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/378119846/485228802.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/347720188/515827693.10000
 0	0.0
 10	10.0
 100	200.0
@@ -448,7 +448,7 @@
 98	196.0
 query: SELECT DEST2.* FROM DEST2
 Input: default/dest2
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/887469224/901850559.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/241813164/313890379.10000
 0	0.0
 10	10.0
 100	200.0

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8.q.out Thu Jun  4 01:21:30 2009
@@ -59,19 +59,19 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/560077282/216523586.10004 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: bigint
+        /data/users/njain/hive2/hive2/build/ql/tmp/140111117/49441246.10004 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: bigint
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -120,20 +120,20 @@
   Stage: Stage-3
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/560077282/216523586.10005 
-          Reduce Output Operator
-            key expressions:
-                  expr: key
-                  type: string
-                  expr: substr(value, 5)
-                  type: string
-            sort order: ++
-            Map-reduce partition columns:
-                  expr: key
-                  type: string
-                  expr: substr(value, 5)
-                  type: string
-            tag: -1
+        /data/users/njain/hive2/hive2/build/ql/tmp/140111117/49441246.10005 
+            Reduce Output Operator
+              key expressions:
+                    expr: key
+                    type: string
+                    expr: substr(value, 5)
+                    type: string
+              sort order: ++
+              Map-reduce partition columns:
+                    expr: key
+                    type: string
+                    expr: substr(value, 5)
+                    type: string
+              tag: -1
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -153,19 +153,19 @@
   Stage: Stage-4
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/560077282/216523586.10006 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: bigint
+        /data/users/njain/hive2/hive2/build/ql/tmp/140111117/49441246.10006 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: bigint
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -204,7 +204,7 @@
 Output: default/dest2
 query: SELECT DEST1.* FROM DEST1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/295910206/467633574.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1852150935/396027054.10000
 0	1
 10	1
 100	1
@@ -516,7 +516,7 @@
 98	1
 query: SELECT DEST2.* FROM DEST2
 Input: default/dest2
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/126981477/158731393.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/17802714/1443308332.10000
 0	1
 10	1
 100	1

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map.q.out Thu Jun  4 01:21:30 2009
@@ -105,21 +105,21 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/1049333631/1099366906.10004 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-                  expr: 1
-                  type: string
-            sort order: ++
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 2
-                  type: bigint
+        /data/users/njain/hive2/hive2/build/ql/tmp/492007815/594155621.10004 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+                    expr: 1
+                    type: string
+              sort order: ++
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 2
+                    type: bigint
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -158,7 +158,7 @@
 Output: default/dest2
 query: SELECT DEST1.* FROM DEST1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/1585620805/348095372.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1292699155/23668440.10000
 0	1
 10	1
 100	1
@@ -470,7 +470,7 @@
 98	1
 query: SELECT DEST2.* FROM DEST2
 Input: default/dest2
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/12995442/81385789.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/1153711442/698935476.10000
 0	1
 10	1
 100	1

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map_skew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map_skew.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map_skew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map_skew.q.out Thu Jun  4 01:21:30 2009
@@ -80,19 +80,19 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/274046296/767130723.10004 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: bigint
+        /data/users/njain/hive2/hive2/build/ql/tmp/409785616/966909649.10004 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: bigint
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -141,23 +141,23 @@
   Stage: Stage-3
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/274046296/767130723.10005 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-                  expr: 1
-                  type: string
-            sort order: ++
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-                  expr: 1
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 2
-                  type: bigint
+        /data/users/njain/hive2/hive2/build/ql/tmp/409785616/966909649.10005 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+                    expr: 1
+                    type: string
+              sort order: ++
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+                    expr: 1
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 2
+                    type: bigint
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -177,19 +177,19 @@
   Stage: Stage-4
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/274046296/767130723.10006 
-          Reduce Output Operator
-            key expressions:
-                  expr: 0
-                  type: string
-            sort order: +
-            Map-reduce partition columns:
-                  expr: 0
-                  type: string
-            tag: -1
-            value expressions:
-                  expr: 1
-                  type: bigint
+        /data/users/njain/hive2/hive2/build/ql/tmp/409785616/966909649.10006 
+            Reduce Output Operator
+              key expressions:
+                    expr: 0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: 0
+                    type: string
+              tag: -1
+              value expressions:
+                    expr: 1
+                    type: bigint
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -228,7 +228,7 @@
 Output: default/dest2
 query: SELECT DEST1.* FROM DEST1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/589113181/576499023.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/76628135/665663173.10000
 0	1
 10	1
 100	1
@@ -540,7 +540,7 @@
 98	1
 query: SELECT DEST2.* FROM DEST2
 Input: default/dest2
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/989060991/1124156593.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/694888471/362559158.10000
 0	1
 10	1
 100	1

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_noskew.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_noskew.q.out?rev=781633&r1=781632&r2=781633&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_noskew.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_noskew.q.out Thu Jun  4 01:21:30 2009
@@ -84,18 +84,18 @@
   Stage: Stage-2
     Map Reduce
       Alias -> Map Operator Tree:
-        /data/users/athusoo/commits/hive_trunk_ws8/build/ql/tmp/280831989/373656.10004 
-          Reduce Output Operator
-            key expressions:
-                  expr: key
-                  type: string
-                  expr: substr(value, 5)
-                  type: string
-            sort order: ++
-            Map-reduce partition columns:
-                  expr: key
-                  type: string
-            tag: -1
+        /data/users/njain/hive2/hive2/build/ql/tmp/8527497/734265590.10004 
+            Reduce Output Operator
+              key expressions:
+                    expr: key
+                    type: string
+                    expr: substr(value, 5)
+                    type: string
+              sort order: ++
+              Map-reduce partition columns:
+                    expr: key
+                    type: string
+              tag: -1
       Reduce Operator Tree:
         Group By Operator
           aggregations:
@@ -134,7 +134,7 @@
 Output: default/dest2
 query: SELECT DEST1.* FROM DEST1
 Input: default/dest1
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/14233066/985088079.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/66885571/152113333.10000
 0	1
 10	1
 100	1
@@ -446,7 +446,7 @@
 98	1
 query: SELECT DEST2.* FROM DEST2
 Input: default/dest2
-Output: /data/users/athusoo/commits/hive_trunk_ws8/ql/../build/ql/tmp/1779608708/316437586.10000
+Output: /data/users/njain/hive2/hive2/ql/../build/ql/tmp/10910469/133549655.10000
 0	1
 10	1
 100	1